paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()),
+ taskExecutionContext.getDefinedParams(),
+ pythonParameters.getLocalParametersMap(),
+ CommandType.of(taskExecutionContext.getCmdTypeIfComplement()),
+ taskExecutionContext.getScheduleTime());
+
+ try {
+ rawPythonScript = VarPoolUtils.convertPythonScriptPlaceholders(rawPythonScript);
+ }
+ catch (StringIndexOutOfBoundsException e) {
+ logger.error("setShareVar field format error, raw python script : {}", rawPythonScript);
+ }
+
+ if (paramsMap != null) {
+ rawPythonScript = ParameterUtils.convertParameterPlaceholders(rawPythonScript, ParamUtils.convert(paramsMap));
+ }
+
+ logger.info("raw python script : {}", pythonParameters.getRawScript());
+ logger.info("task dir : {}", taskDir);
+
+ return rawPythonScript;
+ }
+ @Override
+ public AbstractParameters getParameters() {
+ return pythonParameters;
+ }
+
}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
index 6abb381583..49bfb5f9a8 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
@@ -20,6 +20,7 @@ import org.apache.commons.lang.StringUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
+
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.ZKNodeType;
@@ -33,6 +34,7 @@ import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@@ -45,309 +47,309 @@ import static org.apache.dolphinscheduler.common.Constants.SLEEP_TIME_MILLIS;
/**
- * zookeeper master client
- *
- * single instance
+ * zookeeper master client
+ *
+ * single instance
*/
@Component
public class ZKMasterClient extends AbstractZKClient {
- /**
- * logger
- */
- private static final Logger logger = LoggerFactory.getLogger(ZKMasterClient.class);
-
- /**
- * process service
- */
- @Autowired
- private ProcessService processService;
-
- public void start() {
-
- InterProcessMutex mutex = null;
- try {
- // create distributed lock with the root node path of the lock space as /dolphinscheduler/lock/failover/master
- String znodeLock = getMasterStartUpLockPath();
- mutex = new InterProcessMutex(getZkClient(), znodeLock);
- mutex.acquire();
-
- // init system znode
- this.initSystemZNode();
-
- while (!checkZKNodeExists(NetUtils.getHost(), ZKNodeType.MASTER)){
- ThreadUtils.sleep(SLEEP_TIME_MILLIS);
- }
-
-
- // self tolerant
- if (getActiveMasterNum() == 1) {
- failoverWorker(null, true);
- failoverMaster(null);
- }
-
- }catch (Exception e){
- logger.error("master start up exception",e);
- }finally {
- releaseMutex(mutex);
- }
- }
-
- @Override
- public void close(){
- super.close();
- }
-
- /**
- * handle path events that this class cares about
- * @param client zkClient
- * @param event path event
- * @param path zk path
- */
- @Override
- protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) {
- //monitor master
- if(path.startsWith(getZNodeParentPath(ZKNodeType.MASTER)+Constants.SINGLE_SLASH)){
- handleMasterEvent(event,path);
- }else if(path.startsWith(getZNodeParentPath(ZKNodeType.WORKER)+Constants.SINGLE_SLASH)){
- //monitor worker
- handleWorkerEvent(event,path);
- }
- }
-
- /**
- * remove zookeeper node path
- *
- * @param path zookeeper node path
- * @param zkNodeType zookeeper node type
- * @param failover is failover
- */
- private void removeZKNodePath(String path, ZKNodeType zkNodeType, boolean failover) {
- logger.info("{} node deleted : {}", zkNodeType.toString(), path);
- InterProcessMutex mutex = null;
- try {
- String failoverPath = getFailoverLockPath(zkNodeType);
- // create a distributed lock
- mutex = new InterProcessMutex(getZkClient(), failoverPath);
- mutex.acquire();
-
- String serverHost = getHostByEventDataPath(path);
- // handle dead server
- handleDeadServer(path, zkNodeType, Constants.ADD_ZK_OP);
- //failover server
- if(failover){
- failoverServerWhenDown(serverHost, zkNodeType);
- }
- }catch (Exception e){
- logger.error("{} server failover failed.", zkNodeType.toString());
- logger.error("failover exception ",e);
- }
- finally {
- releaseMutex(mutex);
- }
- }
-
- /**
- * failover server when server down
- *
- * @param serverHost server host
- * @param zkNodeType zookeeper node type
- * @throws Exception exception
- */
- private void failoverServerWhenDown(String serverHost, ZKNodeType zkNodeType) throws Exception {
- if(StringUtils.isEmpty(serverHost) || serverHost.startsWith(NetUtils.getHost())){
- return ;
- }
- switch (zkNodeType){
- case MASTER:
- failoverMaster(serverHost);
- break;
- case WORKER:
- failoverWorker(serverHost, true);
- default:
- break;
- }
- }
-
- /**
- * get failover lock path
- *
- * @param zkNodeType zookeeper node type
- * @return fail over lock path
- */
- private String getFailoverLockPath(ZKNodeType zkNodeType){
-
- switch (zkNodeType){
- case MASTER:
- return getMasterFailoverLockPath();
- case WORKER:
- return getWorkerFailoverLockPath();
- default:
- return "";
- }
- }
-
- /**
- * monitor master
- * @param event event
- * @param path path
- */
- public void handleMasterEvent(TreeCacheEvent event, String path){
- switch (event.getType()) {
- case NODE_ADDED:
- logger.info("master node added : {}", path);
- break;
- case NODE_REMOVED:
- removeZKNodePath(path, ZKNodeType.MASTER, true);
- break;
- default:
- break;
- }
- }
-
- /**
- * monitor worker
- * @param event event
- * @param path path
- */
- public void handleWorkerEvent(TreeCacheEvent event, String path){
- switch (event.getType()) {
- case NODE_ADDED:
- logger.info("worker node added : {}", path);
- break;
- case NODE_REMOVED:
- logger.info("worker node deleted : {}", path);
- removeZKNodePath(path, ZKNodeType.WORKER, true);
- break;
- default:
- break;
- }
- }
-
- /**
- * task needs failover if task start before worker starts
- *
- * @param taskInstance task instance
- * @return true if task instance need fail over
- */
- private boolean checkTaskInstanceNeedFailover(TaskInstance taskInstance) throws Exception {
-
- boolean taskNeedFailover = true;
-
- //now no host will execute this task instance,so no need to failover the task
- if(taskInstance.getHost() == null){
- return false;
- }
-
- // if the worker node exists in zookeeper, we must check the task starts after the worker
- if(checkZKNodeExists(taskInstance.getHost(), ZKNodeType.WORKER)){
- //if task start after worker starts, there is no need to failover the task.
- if(checkTaskAfterWorkerStart(taskInstance)){
- taskNeedFailover = false;
- }
- }
- return taskNeedFailover;
- }
-
- /**
- * check task start after the worker server starts.
- *
- * @param taskInstance task instance
- * @return true if task instance start time after worker server start date
- */
- private boolean checkTaskAfterWorkerStart(TaskInstance taskInstance) {
- if(StringUtils.isEmpty(taskInstance.getHost())){
- return false;
- }
- Date workerServerStartDate = null;
- List workerServers = getServersList(ZKNodeType.WORKER);
- for(Server workerServer : workerServers){
- if(taskInstance.getHost().equals(workerServer.getHost() + Constants.COLON + workerServer.getPort())){
- workerServerStartDate = workerServer.getCreateTime();
- break;
- }
- }
-
- if(workerServerStartDate != null){
- return taskInstance.getStartTime().after(workerServerStartDate);
- }else{
- return false;
- }
- }
-
- /**
- * failover worker tasks
- *
- * 1. kill yarn job if there are yarn jobs in tasks.
- * 2. change task state from running to need failover.
- * 3. failover all tasks when workerHost is null
- * @param workerHost worker host
- */
-
- /**
- * failover worker tasks
- *
- * 1. kill yarn job if there are yarn jobs in tasks.
- * 2. change task state from running to need failover.
- * 3. failover all tasks when workerHost is null
- * @param workerHost worker host
- * @param needCheckWorkerAlive need check worker alive
- * @throws Exception exception
- */
- private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
- logger.info("start worker[{}] failover ...", workerHost);
-
- List needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
- for(TaskInstance taskInstance : needFailoverTaskInstanceList){
- if(needCheckWorkerAlive){
- if(!checkTaskInstanceNeedFailover(taskInstance)){
- continue;
- }
- }
-
- ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
- if(processInstance != null){
- taskInstance.setProcessInstance(processInstance);
- }
-
- TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
- .buildTaskInstanceRelatedInfo(taskInstance)
- .buildProcessInstanceRelatedInfo(processInstance)
- .create();
- // only kill yarn job if exists , the local thread has exited
- ProcessUtils.killYarnJob(taskExecutionContext);
-
- taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
- processService.saveTaskInstance(taskInstance);
- }
- logger.info("end worker[{}] failover ...", workerHost);
- }
-
- /**
- * failover master tasks
- *
- * @param masterHost master host
- */
- private void failoverMaster(String masterHost) {
- logger.info("start master failover ...");
-
- List needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
-
- //updateProcessInstance host is null and insert into command
- for(ProcessInstance processInstance : needFailoverProcessInstanceList){
- if(Constants.NULL.equals(processInstance.getHost()) ){
- continue;
- }
- processService.processNeedFailoverProcessInstances(processInstance);
- }
-
- logger.info("master failover end");
- }
-
- public InterProcessMutex blockAcquireMutex() throws Exception {
- InterProcessMutex mutex = new InterProcessMutex(getZkClient(), getMasterLockPath());
- mutex.acquire();
- return mutex;
- }
-
+ /**
+ * logger
+ */
+ private static final Logger logger = LoggerFactory.getLogger(ZKMasterClient.class);
+
+ /**
+ * process service
+ */
+ @Autowired
+ private ProcessService processService;
+
+ public void start() {
+
+ InterProcessMutex mutex = null;
+ try {
+ // create distributed lock with the root node path of the lock space as /dolphinscheduler/lock/failover/master
+ String znodeLock = getMasterStartUpLockPath();
+ mutex = new InterProcessMutex(getZkClient(), znodeLock);
+ mutex.acquire();
+
+ // init system znode
+ this.initSystemZNode();
+
+ while (!checkZKNodeExists(NetUtils.getHost(), ZKNodeType.MASTER)) {
+ ThreadUtils.sleep(SLEEP_TIME_MILLIS);
+ }
+
+ // self tolerant
+ if (getActiveMasterNum() == 1) {
+ failoverWorker(null, true);
+ failoverMaster(null);
+ }
+
+ } catch (Exception e) {
+ logger.error("master start up exception", e);
+ } finally {
+ releaseMutex(mutex);
+ }
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ }
+
+ /**
+ * handle path events that this class cares about
+ *
+ * @param client zkClient
+ * @param event path event
+ * @param path zk path
+ */
+ @Override
+ protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) {
+ //monitor master
+ if (path.startsWith(getZNodeParentPath(ZKNodeType.MASTER) + Constants.SINGLE_SLASH)) {
+ handleMasterEvent(event, path);
+ } else if (path.startsWith(getZNodeParentPath(ZKNodeType.WORKER) + Constants.SINGLE_SLASH)) {
+ //monitor worker
+ handleWorkerEvent(event, path);
+ }
+ }
+
+ /**
+ * remove zookeeper node path
+ *
+ * @param path zookeeper node path
+ * @param zkNodeType zookeeper node type
+ * @param failover is failover
+ */
+ private void removeZKNodePath(String path, ZKNodeType zkNodeType, boolean failover) {
+ logger.info("{} node deleted : {}", zkNodeType.toString(), path);
+ InterProcessMutex mutex = null;
+ try {
+ String failoverPath = getFailoverLockPath(zkNodeType);
+ // create a distributed lock
+ mutex = new InterProcessMutex(getZkClient(), failoverPath);
+ mutex.acquire();
+
+ String serverHost = getHostByEventDataPath(path);
+ // handle dead server
+ handleDeadServer(path, zkNodeType, Constants.ADD_ZK_OP);
+ //failover server
+ if (failover) {
+ failoverServerWhenDown(serverHost, zkNodeType);
+ }
+ } catch (Exception e) {
+ logger.error("{} server failover failed.", zkNodeType.toString());
+ logger.error("failover exception ", e);
+ } finally {
+ releaseMutex(mutex);
+ }
+ }
+
+ /**
+ * failover server when server down
+ *
+ * @param serverHost server host
+ * @param zkNodeType zookeeper node type
+ * @throws Exception exception
+ */
+ private void failoverServerWhenDown(String serverHost, ZKNodeType zkNodeType) throws Exception {
+ if (StringUtils.isEmpty(serverHost)) {
+ return;
+ }
+ switch (zkNodeType) {
+ case MASTER:
+ failoverMaster(serverHost);
+ break;
+ case WORKER:
+ failoverWorker(serverHost, true);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * get failover lock path
+ *
+ * @param zkNodeType zookeeper node type
+ * @return fail over lock path
+ */
+ private String getFailoverLockPath(ZKNodeType zkNodeType) {
+
+ switch (zkNodeType) {
+ case MASTER:
+ return getMasterFailoverLockPath();
+ case WORKER:
+ return getWorkerFailoverLockPath();
+ default:
+ return "";
+ }
+ }
+
+ /**
+ * monitor master
+ *
+ * @param event event
+ * @param path path
+ */
+ public void handleMasterEvent(TreeCacheEvent event, String path) {
+ switch (event.getType()) {
+ case NODE_ADDED:
+ logger.info("master node added : {}", path);
+ break;
+ case NODE_REMOVED:
+ removeZKNodePath(path, ZKNodeType.MASTER, true);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * monitor worker
+ *
+ * @param event event
+ * @param path path
+ */
+ public void handleWorkerEvent(TreeCacheEvent event, String path) {
+ switch (event.getType()) {
+ case NODE_ADDED:
+ logger.info("worker node added : {}", path);
+ break;
+ case NODE_REMOVED:
+ logger.info("worker node deleted : {}", path);
+ removeZKNodePath(path, ZKNodeType.WORKER, true);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * task needs failover if task start before worker starts
+ *
+ * @param taskInstance task instance
+ * @return true if task instance need fail over
+ */
+ private boolean checkTaskInstanceNeedFailover(TaskInstance taskInstance) throws Exception {
+
+ boolean taskNeedFailover = true;
+
+ //now no host will execute this task instance,so no need to failover the task
+ if (taskInstance.getHost() == null) {
+ return false;
+ }
+
+ // if the worker node exists in zookeeper, we must check the task starts after the worker
+ if (checkZKNodeExists(taskInstance.getHost(), ZKNodeType.WORKER)) {
+ //if task start after worker starts, there is no need to failover the task.
+ if (checkTaskAfterWorkerStart(taskInstance)) {
+ taskNeedFailover = false;
+ }
+ }
+ return taskNeedFailover;
+ }
+
+ /**
+ * check task start after the worker server starts.
+ *
+ * @param taskInstance task instance
+ * @return true if task instance start time after worker server start date
+ */
+ private boolean checkTaskAfterWorkerStart(TaskInstance taskInstance) {
+ if (StringUtils.isEmpty(taskInstance.getHost())) {
+ return false;
+ }
+ Date workerServerStartDate = null;
+ List workerServers = getServersList(ZKNodeType.WORKER);
+ for (Server workerServer : workerServers) {
+ if (taskInstance.getHost().equals(workerServer.getHost() + Constants.COLON + workerServer.getPort())) {
+ workerServerStartDate = workerServer.getCreateTime();
+ break;
+ }
+ }
+ if (workerServerStartDate != null) {
+ return taskInstance.getStartTime().after(workerServerStartDate);
+ }
+ return false;
+ }
+
+ /**
+ * failover worker tasks
+ *
+ * 1. kill yarn job if there are yarn jobs in tasks.
+ * 2. change task state from running to need failover.
+ * 3. failover all tasks when workerHost is null
+ * @param workerHost worker host
+ */
+
+ /**
+ * failover worker tasks
+ *
+ * 1. kill yarn job if there are yarn jobs in tasks.
+ * 2. change task state from running to need failover.
+ * 3. failover all tasks when workerHost is null
+ *
+ * @param workerHost worker host
+ * @param needCheckWorkerAlive need check worker alive
+ * @throws Exception exception
+ */
+ private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
+ logger.info("start worker[{}] failover ...", workerHost);
+
+ List needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
+ for (TaskInstance taskInstance : needFailoverTaskInstanceList) {
+ if (needCheckWorkerAlive) {
+ if (!checkTaskInstanceNeedFailover(taskInstance)) {
+ continue;
+ }
+ }
+
+ ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
+ if (processInstance != null) {
+ taskInstance.setProcessInstance(processInstance);
+ }
+
+ TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
+ .buildTaskInstanceRelatedInfo(taskInstance)
+ .buildProcessInstanceRelatedInfo(processInstance)
+ .create();
+ // only kill yarn job if exists , the local thread has exited
+ ProcessUtils.killYarnJob(taskExecutionContext);
+
+ taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
+ processService.saveTaskInstance(taskInstance);
+ }
+ logger.info("end worker[{}] failover ...", workerHost);
+ }
+
+ /**
+ * failover master tasks
+ *
+ * @param masterHost master host
+ */
+ private void failoverMaster(String masterHost) {
+ logger.info("start master failover ...");
+
+ List needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
+
+ //updateProcessInstance host is null and insert into command
+ for (ProcessInstance processInstance : needFailoverProcessInstanceList) {
+ if (Constants.NULL.equals(processInstance.getHost())) {
+ continue;
+ }
+ processService.processNeedFailoverProcessInstances(processInstance);
+ }
+
+ logger.info("master failover end");
+ }
+
+ public InterProcessMutex blockAcquireMutex() throws Exception {
+ InterProcessMutex mutex = new InterProcessMutex(getZkClient(), getMasterLockPath());
+ mutex.acquire();
+ return mutex;
+ }
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
index fadaa84a69..fd5dda0873 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
@@ -14,9 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import org.apache.dolphinscheduler.remote.utils.Host;
+
+import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
@@ -27,15 +30,47 @@ public class LowerWeightRoundRobinTest {
@Test
- public void testSelect(){
+ public void testSelect() {
Collection sources = new ArrayList<>();
- sources.add(new HostWeight(Host.of("192.158.2.1:11"), 0.06, 0.44, 3.84));
- sources.add(new HostWeight(Host.of("192.158.2.1:22"), 0.06, 0.56, 3.24));
- sources.add(new HostWeight(Host.of("192.158.2.1:33"), 0.06, 0.80, 3.15));
- System.out.println(sources);
+ sources.add(new HostWeight(Host.of("192.158.2.1:11:100:" + (System.currentTimeMillis() - 60 * 8 * 1000)), 0.06, 0.44, 3.84));
+ sources.add(new HostWeight(Host.of("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)), 0.06, 0.56, 3.24));
+ sources.add(new HostWeight(Host.of("192.158.2.3:33:100:" + (System.currentTimeMillis() - 60 * 2 * 1000)), 0.06, 0.80, 3.15));
+
+ LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
+ HostWeight result;
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.1", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.2", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.1", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.2", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.1", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.2", result.getHost().getIp());
+ }
+
+ @Test
+ public void testWarmUpSelect() {
+ Collection sources = new ArrayList<>();
+ sources.add(new HostWeight(Host.of("192.158.2.1:11:100:" + (System.currentTimeMillis() - 60 * 8 * 1000)), 0.06, 0.44, 3.84));
+ sources.add(new HostWeight(Host.of("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)), 0.06, 0.44, 3.84));
+ sources.add(new HostWeight(Host.of("192.158.2.3:33:100:" + (System.currentTimeMillis() - 60 * 3 * 1000)), 0.06, 0.44, 3.84));
+ sources.add(new HostWeight(Host.of("192.158.2.4:33:100:" + (System.currentTimeMillis() - 60 * 11 * 1000)), 0.06, 0.44, 3.84));
+
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
- for(int i = 0; i < 100; i ++){
- System.out.println(roundRobin.select(sources));
- }
+ HostWeight result;
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.4", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.1", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.2", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.4", result.getHost().getIp());
+ result = roundRobin.select(sources);
+ Assert.assertEquals("192.158.2.1", result.getHost().getIp());
}
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
index f25a227947..14aa7b8f1f 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
@@ -14,11 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
-import org.apache.commons.lang.ObjectUtils;
-import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.utils.Host;
+
import org.junit.Assert;
import org.junit.Test;
@@ -31,22 +31,22 @@ import java.util.Collections;
public class RandomSelectorTest {
@Test(expected = IllegalArgumentException.class)
- public void testSelectWithIllegalArgumentException(){
+ public void testSelectWithIllegalArgumentException() {
RandomSelector selector = new RandomSelector();
- selector.select(Collections.EMPTY_LIST);
+ selector.select(null);
}
@Test
- public void testSelect1(){
+ public void testSelect1() {
RandomSelector selector = new RandomSelector();
- Host result = selector.select(Arrays.asList(new Host("192.168.1.1",80,100),new Host("192.168.1.2",80,20)));
+ Host result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 100, System.currentTimeMillis()), new Host("192.168.1.2", 80, 20, System.currentTimeMillis())));
Assert.assertNotNull(result);
}
@Test
- public void testSelect(){
+ public void testSelect() {
RandomSelector selector = new RandomSelector();
- Host result = selector.select(Arrays.asList(new Host("192.168.1.1",80,100),new Host("192.168.1.1",80,20)));
+ Host result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 100, System.currentTimeMillis()), new Host("192.168.1.1", 80, 20, System.currentTimeMillis())));
Assert.assertNotNull(result);
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
index ed62caaa2c..9e41cd68bf 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
@@ -14,16 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
-import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.utils.Host;
+
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
-import java.util.List;
/**
* round robin selector
@@ -33,43 +33,59 @@ public class RoundRobinSelectorTest {
@Test(expected = IllegalArgumentException.class)
public void testSelectWithIllegalArgumentException() {
RoundRobinSelector selector = new RoundRobinSelector();
- selector.select(Collections.EMPTY_LIST);
+ selector.select(null);
}
@Test
public void testSelect1() {
RoundRobinSelector selector = new RoundRobinSelector();
- Host result = null;
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
+ Host result;
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
// add new host
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.1", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.2", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.3", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.1", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.1",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.3",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.1",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.2",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.1",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.3",result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.1", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
+ new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.3", result.getIp());
// remove host3
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.1",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.2",result.getIp());
- result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
- Assert.assertEquals("192.168.1.1",result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.1", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.2", result.getIp());
+ result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
+ Assert.assertEquals("192.168.1.1", result.getIp());
+
+ }
+
+ @Test
+ public void testWarmUpRoundRobinSelector() {
+ RoundRobinSelector selector = new RoundRobinSelector();
+ Host result;
+ result = selector.select(
+ Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis() - 60 * 1000 * 2, "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis() - 60 * 1000 * 10, "kris")));
+ Assert.assertEquals("192.168.1.2", result.getIp());
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/HostTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/HostTest.java
new file mode 100644
index 0000000000..6273569485
--- /dev/null
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/HostTest.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.utils;
+
+import org.apache.dolphinscheduler.remote.utils.Host;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * host test
+ */
+public class HostTest {
+
+ @Test
+ public void testHostWarmUp() {
+ Host host = Host.of(("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)));
+ Assert.assertEquals(50, host.getWeight());
+ host = Host.of(("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 10 * 1000)));
+ Assert.assertEquals(100, host.getWeight());
+ }
+
+ @Test
+ public void testHost() {
+ Host host = Host.of("192.158.2.2:22");
+ Assert.assertEquals(22, host.getPort());
+ }
+}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/TaskManagerTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/TaskManagerTest.java
new file mode 100644
index 0000000000..eb0383979c
--- /dev/null
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/TaskManagerTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.worker.task;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
+import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+
+import java.util.Date;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({SpringApplicationContext.class})
+public class TaskManagerTest {
+
+ private static Logger logger = LoggerFactory.getLogger(TaskManagerTest.class);
+
+ private TaskExecutionContext taskExecutionContext;
+
+ private Logger taskLogger;
+
+ private TaskExecutionContextCacheManagerImpl taskExecutionContextCacheManager;
+
+ @Before
+ public void before() {
+ // init task execution context, logger
+ taskExecutionContext = new TaskExecutionContext();
+ taskExecutionContext.setProcessId(12345);
+ taskExecutionContext.setProcessDefineId(1);
+ taskExecutionContext.setProcessInstanceId(1);
+ taskExecutionContext.setTaskInstanceId(1);
+ taskExecutionContext.setTaskType("");
+ taskExecutionContext.setFirstSubmitTime(new Date());
+ taskExecutionContext.setDelayTime(0);
+ taskExecutionContext.setLogPath("/tmp/test.log");
+ taskExecutionContext.setHost("localhost");
+ taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/exec/process/1/2/3/4");
+
+ taskLogger = LoggerFactory.getLogger(LoggerUtils.buildTaskId(
+ LoggerUtils.TASK_LOGGER_INFO_PREFIX,
+ taskExecutionContext.getProcessDefineId(),
+ taskExecutionContext.getProcessInstanceId(),
+ taskExecutionContext.getTaskInstanceId()
+ ));
+
+ taskExecutionContextCacheManager = new TaskExecutionContextCacheManagerImpl();
+ taskExecutionContextCacheManager.cacheTaskExecutionContext(taskExecutionContext);
+
+ PowerMockito.mockStatic(SpringApplicationContext.class);
+ PowerMockito.when(SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class))
+ .thenReturn(taskExecutionContextCacheManager);
+ }
+
+ @Test
+ public void testNewTask() {
+
+ taskExecutionContext.setTaskType("SHELL");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("WATERDROP");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("HTTP");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("MR");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("SPARK");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("FLINK");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("PYTHON");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("DATAX");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+ taskExecutionContext.setTaskType("SQOOP");
+ Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
+
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNewTaskIsNull() {
+ taskExecutionContext.setTaskType(null);
+ TaskManager.newTask(taskExecutionContext,taskLogger);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNewTaskIsNotExists() {
+ taskExecutionContext.setTaskType("XXX");
+ TaskManager.newTask(taskExecutionContext,taskLogger);
+ }
+}
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
index 6f642672cd..10069aea10 100644
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
@@ -14,6 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.dolphinscheduler.service.process;
import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE;
@@ -144,8 +145,6 @@ public class ProcessService {
@Autowired
private ResourceMapper resourceMapper;
-
-
@Autowired
private ErrorCommandMapper errorCommandMapper;
@@ -167,12 +166,12 @@ public class ProcessService {
public ProcessInstance handleCommand(Logger logger, String host, int validThreadNum, Command command) {
ProcessInstance processInstance = constructProcessInstance(command, host);
//cannot construct process instance, return null;
- if(processInstance == null){
+ if (processInstance == null) {
logger.error("scan command, command parameter is error: {}", command);
moveToErrorCommand(command, "process instance is null");
return null;
}
- if(!checkThreadNum(command, validThreadNum)){
+ if (!checkThreadNum(command, validThreadNum)) {
logger.info("there is not enough thread for this command: {}", command);
return setWaitingThreadProcess(command, processInstance);
}
@@ -180,7 +179,7 @@ public class ProcessService {
processInstance.addHistoryCmd(command.getCommandType());
saveProcessInstance(processInstance);
this.setSubProcessParam(processInstance);
- delCommandByid(command.getId());
+ delCommandById(command.getId());
return processInstance;
}
@@ -193,7 +192,7 @@ public class ProcessService {
public void moveToErrorCommand(Command command, String message) {
ErrorCommand errorCommand = new ErrorCommand(command, message);
this.errorCommandMapper.insert(errorCommand);
- delCommandByid(command.getId());
+ delCommandById(command.getId());
}
/**
@@ -204,7 +203,7 @@ public class ProcessService {
*/
private ProcessInstance setWaitingThreadProcess(Command command, ProcessInstance processInstance) {
processInstance.setState(ExecutionStatus.WAITTING_THREAD);
- if(command.getCommandType() != CommandType.RECOVER_WAITTING_THREAD){
+ if (command.getCommandType() != CommandType.RECOVER_WAITTING_THREAD) {
processInstance.addHistoryCmd(command.getCommandType());
}
saveProcessInstance(processInstance);
@@ -231,7 +230,7 @@ public class ProcessService {
*/
public int createCommand(Command command) {
int result = 0;
- if (command != null){
+ if (command != null) {
result = commandMapper.insert(command);
}
return result;
@@ -241,7 +240,7 @@ public class ProcessService {
* find one command from queue list
* @return command
*/
- public Command findOneCommand(){
+ public Command findOneCommand() {
return commandMapper.getOneToRun();
}
@@ -250,7 +249,7 @@ public class ProcessService {
* @param command command
* @return create command result
*/
- public Boolean verifyIsNeedCreateCommand(Command command){
+ public Boolean verifyIsNeedCreateCommand(Command command) {
Boolean isNeedCreate = true;
Map cmdTypeMap = new HashMap();
cmdTypeMap.put(CommandType.REPEAT_RUNNING,1);
@@ -258,16 +257,16 @@ public class ProcessService {
cmdTypeMap.put(CommandType.START_FAILURE_TASK_PROCESS,1);
CommandType commandType = command.getCommandType();
- if(cmdTypeMap.containsKey(commandType)){
+ if (cmdTypeMap.containsKey(commandType)) {
ObjectNode cmdParamObj = JSONUtils.parseObject(command.getCommandParam());
int processInstanceId = cmdParamObj.path(CMDPARAM_RECOVER_PROCESS_ID_STRING).asInt();
List commands = commandMapper.selectList(null);
// for all commands
- for (Command tmpCommand:commands){
- if(cmdTypeMap.containsKey(tmpCommand.getCommandType())){
+ for (Command tmpCommand:commands) {
+ if (cmdTypeMap.containsKey(tmpCommand.getCommandType())) {
ObjectNode tempObj = JSONUtils.parseObject(tmpCommand.getCommandParam());
- if(tempObj != null && processInstanceId == tempObj.path(CMDPARAM_RECOVER_PROCESS_ID_STRING).asInt()){
+ if (tempObj != null && processInstanceId == tempObj.path(CMDPARAM_RECOVER_PROCESS_ID_STRING).asInt()) {
isNeedCreate = false;
break;
}
@@ -282,7 +281,7 @@ public class ProcessService {
* @param processId processId
* @return process instance
*/
- public ProcessInstance findProcessInstanceDetailById(int processId){
+ public ProcessInstance findProcessInstanceDetailById(int processId) {
return processInstanceMapper.queryDetailById(processId);
}
@@ -291,7 +290,7 @@ public class ProcessService {
* @param defineId
* @return
*/
- public List getTaskNodeListByDefinitionId(Integer defineId){
+ public List getTaskNodeListByDefinitionId(Integer defineId) {
ProcessDefinition processDefinition = processDefineMapper.selectById(defineId);
if (processDefinition == null) {
logger.info("process define not exists");
@@ -315,7 +314,7 @@ public class ProcessService {
* @param processId processId
* @return process instance
*/
- public ProcessInstance findProcessInstanceById(int processId){
+ public ProcessInstance findProcessInstanceById(int processId) {
return processInstanceMapper.selectById(processId);
}
@@ -333,7 +332,7 @@ public class ProcessService {
* @param processInstanceId processInstanceId
* @return delete process instance result
*/
- public int deleteWorkProcessInstanceById(int processInstanceId){
+ public int deleteWorkProcessInstanceById(int processInstanceId) {
return processInstanceMapper.deleteById(processInstanceId);
}
@@ -342,11 +341,11 @@ public class ProcessService {
* @param processInstanceId processInstanceId
* @return delete all sub process instance result
*/
- public int deleteAllSubWorkProcessByParentId(int processInstanceId){
+ public int deleteAllSubWorkProcessByParentId(int processInstanceId) {
List subProcessIdList = processInstanceMapMapper.querySubIdListByParentId(processInstanceId);
- for(Integer subId : subProcessIdList){
+ for (Integer subId : subProcessIdList) {
deleteAllSubWorkProcessByParentId(subId);
deleteWorkProcessMapByParentId(subId);
removeTaskLogFile(subId);
@@ -355,51 +354,48 @@ public class ProcessService {
return 1;
}
-
/**
* remove task log file
* @param processInstanceId processInstanceId
*/
- public void removeTaskLogFile(Integer processInstanceId){
+ public void removeTaskLogFile(Integer processInstanceId) {
LogClientService logClient = new LogClientService();
List taskInstanceList = findValidTaskListByProcessId(processInstanceId);
- if (CollectionUtils.isEmpty(taskInstanceList)){
+ if (CollectionUtils.isEmpty(taskInstanceList)) {
return;
}
- for (TaskInstance taskInstance : taskInstanceList){
+ for (TaskInstance taskInstance : taskInstanceList) {
String taskLogPath = taskInstance.getLogPath();
- if (StringUtils.isEmpty(taskInstance.getHost())){
+ if (StringUtils.isEmpty(taskInstance.getHost())) {
continue;
}
int port = Constants.RPC_PORT;
String ip = "";
try {
ip = Host.of(taskInstance.getHost()).getIp();
- }catch (Exception e){
+ } catch (Exception e) {
// compatible old version
ip = taskInstance.getHost();
}
-
// remove task log from loggerserver
logClient.removeTaskLog(ip,port,taskLogPath);
}
}
-
/**
* calculate sub process number in the process define.
* @param processDefinitionId processDefinitionId
* @return process thread num count
*/
- private Integer workProcessThreadNumCount(Integer processDefinitionId){
+ private Integer workProcessThreadNumCount(Integer processDefinitionId) {
List ids = new ArrayList<>();
recurseFindSubProcessId(processDefinitionId, ids);
- return ids.size()+1;
+ return ids.size() + 1;
}
/**
@@ -407,7 +403,7 @@ public class ProcessService {
* @param parentId parentId
* @param ids ids
*/
- public void recurseFindSubProcessId(int parentId, List ids){
+ public void recurseFindSubProcessId(int parentId, List ids) {
ProcessDefinition processDefinition = processDefineMapper.selectById(parentId);
String processDefinitionJson = processDefinition.getProcessDefinitionJson();
@@ -415,12 +411,12 @@ public class ProcessService {
List taskNodeList = processData.getTasks();
- if (taskNodeList != null && taskNodeList.size() > 0){
+ if (taskNodeList != null && taskNodeList.size() > 0) {
- for (TaskNode taskNode : taskNodeList){
+ for (TaskNode taskNode : taskNodeList) {
String parameter = taskNode.getParams();
ObjectNode parameterJson = JSONUtils.parseObject(parameter);
- if (parameterJson.get(CMDPARAM_SUB_PROCESS_DEFINE_ID) != null){
+ if (parameterJson.get(CMDPARAM_SUB_PROCESS_DEFINE_ID) != null) {
SubProcessParameters subProcessParam = JSONUtils.parseObject(parameter, SubProcessParameters.class);
ids.add(subProcessParam.getProcessDefinitionId());
recurseFindSubProcessId(subProcessParam.getProcessDefinitionId(),ids);
@@ -441,8 +437,8 @@ public class ProcessService {
public void createRecoveryWaitingThreadCommand(Command originCommand, ProcessInstance processInstance) {
// sub process doesnot need to create wait command
- if(processInstance.getIsSubProcess() == Flag.YES){
- if(originCommand != null){
+ if (processInstance.getIsSubProcess() == Flag.YES) {
+ if (originCommand != null) {
commandMapper.deleteById(originCommand.getId());
}
return;
@@ -450,7 +446,7 @@ public class ProcessService {
Map cmdParam = new HashMap<>();
cmdParam.put(Constants.CMDPARAM_RECOVERY_WAITTING_THREAD, String.valueOf(processInstance.getId()));
// process instance quit by "waiting thread" state
- if(originCommand == null){
+ if (originCommand == null) {
Command command = new Command(
CommandType.RECOVER_WAITTING_THREAD,
processInstance.getTaskDependType(),
@@ -464,14 +460,14 @@ public class ProcessService {
processInstance.getProcessInstancePriority()
);
saveCommand(command);
- return ;
+ return;
}
// update the command time if current command if recover from waiting
- if(originCommand.getCommandType() == CommandType.RECOVER_WAITTING_THREAD){
+ if (originCommand.getCommandType() == CommandType.RECOVER_WAITTING_THREAD) {
originCommand.setUpdateTime(new Date());
saveCommand(originCommand);
- }else{
+ } else {
// delete old command and create new waiting thread command
commandMapper.deleteById(originCommand.getId());
originCommand.setId(0);
@@ -489,10 +485,10 @@ public class ProcessService {
* @param cmdParam cmdParam map
* @return date
*/
- private Date getScheduleTime(Command command, Map cmdParam){
+ private Date getScheduleTime(Command command, Map cmdParam) {
Date scheduleTime = command.getScheduleTime();
- if(scheduleTime == null){
- if(cmdParam != null && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)){
+ if (scheduleTime == null) {
+ if (cmdParam != null && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)) {
scheduleTime = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
}
}
@@ -508,7 +504,7 @@ public class ProcessService {
*/
private ProcessInstance generateNewProcessInstance(ProcessDefinition processDefinition,
Command command,
- Map cmdParam){
+ Map cmdParam) {
ProcessInstance processInstance = new ProcessInstance(processDefinition);
processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
processInstance.setRecovery(Flag.NO);
@@ -529,7 +525,7 @@ public class ProcessService {
// schedule time
Date scheduleTime = getScheduleTime(command, cmdParam);
- if(scheduleTime != null){
+ if (scheduleTime != null) {
processInstance.setScheduleTime(scheduleTime);
}
processInstance.setCommandStartTime(command.getStartTime());
@@ -562,17 +558,17 @@ public class ProcessService {
* @param userId userId
* @return tenant
*/
- public Tenant getTenantForProcess(int tenantId, int userId){
+ public Tenant getTenantForProcess(int tenantId, int userId) {
Tenant tenant = null;
- if(tenantId >= 0){
+ if (tenantId >= 0) {
tenant = tenantMapper.queryById(tenantId);
}
- if (userId == 0){
+ if (userId == 0) {
return null;
}
- if(tenant == null){
+ if (tenant == null) {
User user = userMapper.selectById(userId);
tenant = tenantMapper.queryById(user.getTenantId());
}
@@ -585,11 +581,11 @@ public class ProcessService {
* @param cmdParam cmdParam map
* @return whether command param is valid
*/
- private Boolean checkCmdParam(Command command, Map cmdParam){
- if(command.getTaskDependType() == TaskDependType.TASK_ONLY || command.getTaskDependType()== TaskDependType.TASK_PRE){
- if(cmdParam == null
+ private Boolean checkCmdParam(Command command, Map cmdParam) {
+ if (command.getTaskDependType() == TaskDependType.TASK_ONLY || command.getTaskDependType() == TaskDependType.TASK_PRE) {
+ if (cmdParam == null
|| !cmdParam.containsKey(Constants.CMDPARAM_START_NODE_NAMES)
- || cmdParam.get(Constants.CMDPARAM_START_NODE_NAMES).isEmpty()){
+ || cmdParam.get(Constants.CMDPARAM_START_NODE_NAMES).isEmpty()) {
logger.error("command node depend type is {}, but start nodes is null ", command.getTaskDependType());
return false;
}
@@ -603,78 +599,78 @@ public class ProcessService {
* @param host host
* @return process instance
*/
- private ProcessInstance constructProcessInstance(Command command, String host){
+ private ProcessInstance constructProcessInstance(Command command, String host) {
ProcessInstance processInstance = null;
CommandType commandType = command.getCommandType();
Map cmdParam = JSONUtils.toMap(command.getCommandParam());
ProcessDefinition processDefinition = null;
- if(command.getProcessDefinitionId() != 0){
+ if (command.getProcessDefinitionId() != 0) {
processDefinition = processDefineMapper.selectById(command.getProcessDefinitionId());
- if(processDefinition == null){
+ if (processDefinition == null) {
logger.error("cannot find the work process define! define id : {}", command.getProcessDefinitionId());
return null;
}
}
- if(cmdParam != null ){
+ if (cmdParam != null) {
Integer processInstanceId = 0;
// recover from failure or pause tasks
- if(cmdParam.containsKey(Constants.CMDPARAM_RECOVER_PROCESS_ID_STRING)) {
+ if (cmdParam.containsKey(Constants.CMDPARAM_RECOVER_PROCESS_ID_STRING)) {
String processId = cmdParam.get(Constants.CMDPARAM_RECOVER_PROCESS_ID_STRING);
processInstanceId = Integer.parseInt(processId);
if (processInstanceId == 0) {
logger.error("command parameter is error, [ ProcessInstanceId ] is 0");
return null;
}
- }else if(cmdParam.containsKey(Constants.CMDPARAM_SUB_PROCESS)){
+ } else if (cmdParam.containsKey(Constants.CMDPARAM_SUB_PROCESS)) {
// sub process map
String pId = cmdParam.get(Constants.CMDPARAM_SUB_PROCESS);
processInstanceId = Integer.parseInt(pId);
- }else if(cmdParam.containsKey(Constants.CMDPARAM_RECOVERY_WAITTING_THREAD)){
+ } else if (cmdParam.containsKey(Constants.CMDPARAM_RECOVERY_WAITTING_THREAD)) {
// waiting thread command
String pId = cmdParam.get(Constants.CMDPARAM_RECOVERY_WAITTING_THREAD);
processInstanceId = Integer.parseInt(pId);
}
- if(processInstanceId ==0){
+ if (processInstanceId == 0) {
processInstance = generateNewProcessInstance(processDefinition, command, cmdParam);
- }else{
+ } else {
processInstance = this.findProcessInstanceDetailById(processInstanceId);
}
processDefinition = processDefineMapper.selectById(processInstance.getProcessDefinitionId());
processInstance.setProcessDefinition(processDefinition);
//reset command parameter
- if(processInstance.getCommandParam() != null){
+ if (processInstance.getCommandParam() != null) {
Map processCmdParam = JSONUtils.toMap(processInstance.getCommandParam());
- for(Map.Entry entry: processCmdParam.entrySet()) {
- if(!cmdParam.containsKey(entry.getKey())){
+ for (Map.Entry entry: processCmdParam.entrySet()) {
+ if (!cmdParam.containsKey(entry.getKey())) {
cmdParam.put(entry.getKey(), entry.getValue());
}
}
}
// reset command parameter if sub process
- if(cmdParam.containsKey(Constants.CMDPARAM_SUB_PROCESS)){
+ if (cmdParam.containsKey(Constants.CMDPARAM_SUB_PROCESS)) {
processInstance.setCommandParam(command.getCommandParam());
}
- }else{
+ } else {
// generate one new process instance
processInstance = generateNewProcessInstance(processDefinition, command, cmdParam);
}
- if(!checkCmdParam(command, cmdParam)){
+ if (!checkCmdParam(command, cmdParam)) {
logger.error("command parameter check failed!");
return null;
}
- if(command.getScheduleTime() != null){
+ if (command.getScheduleTime() != null) {
processInstance.setScheduleTime(command.getScheduleTime());
}
processInstance.setHost(host);
ExecutionStatus runStatus = ExecutionStatus.RUNNING_EXECUTION;
int runTime = processInstance.getRunTimes();
- switch (commandType){
+ switch (commandType) {
case START_PROCESS:
break;
case START_FAILURE_TASK_PROCESS:
@@ -686,13 +682,13 @@ public class ProcessService {
failedList.addAll(killedList);
failedList.addAll(toleranceList);
- for(Integer taskId : failedList){
+ for (Integer taskId : failedList) {
initTaskInstance(this.findTaskInstanceById(taskId));
}
cmdParam.put(Constants.CMDPARAM_RECOVERY_START_NODE_STRING,
String.join(Constants.COMMA, convertIntListToString(failedList)));
processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam));
- processInstance.setRunTimes(runTime +1 );
+ processInstance.setRunTimes(runTime + 1);
break;
case START_CURRENT_TASK_PROCESS:
break;
@@ -705,13 +701,13 @@ public class ProcessService {
List stopNodeList = findTaskIdByInstanceState(processInstance.getId(),
ExecutionStatus.KILL);
suspendedNodeList.addAll(stopNodeList);
- for(Integer taskId : suspendedNodeList){
+ for (Integer taskId : suspendedNodeList) {
// initialize the pause state
initTaskInstance(this.findTaskInstanceById(taskId));
}
cmdParam.put(Constants.CMDPARAM_RECOVERY_START_NODE_STRING, String.join(",", convertIntListToString(suspendedNodeList)));
processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam));
- processInstance.setRunTimes(runTime +1);
+ processInstance.setRunTimes(runTime + 1);
break;
case RECOVER_TOLERANCE_FAULT_PROCESS:
// recover tolerance fault process
@@ -721,7 +717,7 @@ public class ProcessService {
case COMPLEMENT_DATA:
// delete all the valid tasks when complement data
List taskInstanceList = this.findValidTaskListByProcessId(processInstance.getId());
- for(TaskInstance taskInstance : taskInstanceList){
+ for (TaskInstance taskInstance : taskInstanceList) {
taskInstance.setFlag(Flag.NO);
this.updateTaskInstance(taskInstance);
}
@@ -729,19 +725,19 @@ public class ProcessService {
break;
case REPEAT_RUNNING:
// delete the recover task names from command parameter
- if(cmdParam.containsKey(Constants.CMDPARAM_RECOVERY_START_NODE_STRING)){
+ if (cmdParam.containsKey(Constants.CMDPARAM_RECOVERY_START_NODE_STRING)) {
cmdParam.remove(Constants.CMDPARAM_RECOVERY_START_NODE_STRING);
processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam));
}
// delete all the valid tasks when repeat running
List validTaskList = findValidTaskListByProcessId(processInstance.getId());
- for(TaskInstance taskInstance : validTaskList){
+ for (TaskInstance taskInstance : validTaskList) {
taskInstance.setFlag(Flag.NO);
updateTaskInstance(taskInstance);
}
processInstance.setStartTime(new Date());
processInstance.setEndTime(null);
- processInstance.setRunTimes(runTime +1);
+ processInstance.setRunTimes(runTime + 1);
initComplementDataParam(processDefinition, processInstance, cmdParam);
break;
case SCHEDULER:
@@ -759,10 +755,10 @@ public class ProcessService {
* @param command command
* @return command type
*/
- private CommandType getCommandTypeIfComplement(ProcessInstance processInstance, Command command){
- if(CommandType.COMPLEMENT_DATA == processInstance.getCmdTypeIfComplement()){
+ private CommandType getCommandTypeIfComplement(ProcessInstance processInstance, Command command) {
+ if (CommandType.COMPLEMENT_DATA == processInstance.getCmdTypeIfComplement()) {
return CommandType.COMPLEMENT_DATA;
- }else{
+ } else {
return command.getCommandType();
}
}
@@ -776,13 +772,13 @@ public class ProcessService {
private void initComplementDataParam(ProcessDefinition processDefinition,
ProcessInstance processInstance,
Map cmdParam) {
- if(!processInstance.isComplementData()){
+ if (!processInstance.isComplementData()) {
return;
}
Date startComplementTime = DateUtils.parse(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE),
YYYY_MM_DD_HH_MM_SS);
- if(Flag.NO == processInstance.getIsSubProcess()) {
+ if (Flag.NO == processInstance.getIsSubProcess()) {
processInstance.setScheduleTime(startComplementTime);
}
processInstance.setGlobalParams(ParameterUtils.curingGlobalParams(
@@ -792,7 +788,6 @@ public class ProcessService {
}
-
/**
* set sub work process parameters.
* handle sub work process instance, update relation table and command parameters
@@ -800,15 +795,15 @@ public class ProcessService {
* @param subProcessInstance subProcessInstance
* @return process instance
*/
- public ProcessInstance setSubProcessParam(ProcessInstance subProcessInstance){
+ public ProcessInstance setSubProcessParam(ProcessInstance subProcessInstance) {
String cmdParam = subProcessInstance.getCommandParam();
- if(StringUtils.isEmpty(cmdParam)){
+ if (StringUtils.isEmpty(cmdParam)) {
return subProcessInstance;
}
Map paramMap = JSONUtils.toMap(cmdParam);
// write sub process id into cmd param.
- if(paramMap.containsKey(CMDPARAM_SUB_PROCESS)
- && CMDPARAM_EMPTY_SUB_PROCESS.equals(paramMap.get(CMDPARAM_SUB_PROCESS))){
+ if (paramMap.containsKey(CMDPARAM_SUB_PROCESS)
+ && CMDPARAM_EMPTY_SUB_PROCESS.equals(paramMap.get(CMDPARAM_SUB_PROCESS))) {
paramMap.remove(CMDPARAM_SUB_PROCESS);
paramMap.put(CMDPARAM_SUB_PROCESS, String.valueOf(subProcessInstance.getId()));
subProcessInstance.setCommandParam(JSONUtils.toJsonString(paramMap));
@@ -817,18 +812,18 @@ public class ProcessService {
}
// copy parent instance user def params to sub process..
String parentInstanceId = paramMap.get(CMDPARAM_SUB_PROCESS_PARENT_INSTANCE_ID);
- if(StringUtils.isNotEmpty(parentInstanceId)){
+ if (StringUtils.isNotEmpty(parentInstanceId)) {
ProcessInstance parentInstance = findProcessInstanceDetailById(Integer.parseInt(parentInstanceId));
- if(parentInstance != null){
+ if (parentInstance != null) {
subProcessInstance.setGlobalParams(
joinGlobalParams(parentInstance.getGlobalParams(), subProcessInstance.getGlobalParams()));
this.saveProcessInstance(subProcessInstance);
- }else{
+ } else {
logger.error("sub process command params error, cannot find parent instance: {} ", cmdParam);
}
}
ProcessInstanceMap processInstanceMap = JSONUtils.parseObject(cmdParam, ProcessInstanceMap.class);
- if(processInstanceMap == null || processInstanceMap.getParentProcessInstanceId() == 0){
+ if (processInstanceMap == null || processInstanceMap.getParentProcessInstanceId() == 0) {
return subProcessInstance;
}
// update sub process id to process map table
@@ -845,15 +840,15 @@ public class ProcessService {
* @param subGlobalParams subGlobalParams
* @return global params join
*/
- private String joinGlobalParams(String parentGlobalParams, String subGlobalParams){
+ private String joinGlobalParams(String parentGlobalParams, String subGlobalParams) {
List parentPropertyList = JSONUtils.toList(parentGlobalParams, Property.class);
List subPropertyList = JSONUtils.toList(subGlobalParams, Property.class);
Map subMap = subPropertyList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
- for(Property parent : parentPropertyList){
- if(!subMap.containsKey(parent.getProp())){
+ for (Property parent : parentPropertyList) {
+ if (!subMap.containsKey(parent.getProp())) {
subPropertyList.add(parent);
}
}
@@ -864,10 +859,10 @@ public class ProcessService {
* initialize task instance
* @param taskInstance taskInstance
*/
- private void initTaskInstance(TaskInstance taskInstance){
+ private void initTaskInstance(TaskInstance taskInstance) {
- if(!taskInstance.isSubProcess()){
- if(taskInstance.getState().typeIsCancel() || taskInstance.getState().typeIsFailure()){
+ if (!taskInstance.isSubProcess()) {
+ if (taskInstance.getState().typeIsCancel() || taskInstance.getState().typeIsFailure()) {
taskInstance.setFlag(Flag.NO);
updateTaskInstance(taskInstance);
return;
@@ -884,19 +879,19 @@ public class ProcessService {
* @return task instance
*/
@Transactional(rollbackFor = RuntimeException.class)
- public TaskInstance submitTask(TaskInstance taskInstance){
+ public TaskInstance submitTask(TaskInstance taskInstance) {
ProcessInstance processInstance = this.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
logger.info("start submit task : {}, instance id:{}, state: {}",
taskInstance.getName(), taskInstance.getProcessInstanceId(), processInstance.getState());
//submit to db
TaskInstance task = submitTaskInstanceToDB(taskInstance, processInstance);
- if(task == null){
+ if (task == null) {
logger.error("end submit task to db error, task name:{}, process id:{} state: {} ",
taskInstance.getName(), taskInstance.getProcessInstance(), processInstance.getState());
return task;
}
- if(!task.getState().typeIsFinished()){
- createSubWorkProcessCommand(processInstance, task);
+ if (!task.getState().typeIsFinished()) {
+ createSubWorkProcess(processInstance, task);
}
logger.info("end submit task to db successfully:{} state:{} complete, instance id:{} state: {} ",
@@ -906,20 +901,22 @@ public class ProcessService {
/**
* set work process instance map
+ * consider o
+ * repeat running does not generate new sub process instance
+ * set map {parent instance id, task instance id, 0(child instance id)}
* @param parentInstance parentInstance
* @param parentTask parentTask
* @return process instance map
*/
- private ProcessInstanceMap setProcessInstanceMap(ProcessInstance parentInstance, TaskInstance parentTask){
+ private ProcessInstanceMap setProcessInstanceMap(ProcessInstance parentInstance, TaskInstance parentTask) {
ProcessInstanceMap processMap = findWorkProcessMapByParent(parentInstance.getId(), parentTask.getId());
- if(processMap != null){
+ if (processMap != null) {
return processMap;
- }else if(parentInstance.getCommandType() == CommandType.REPEAT_RUNNING
- || parentInstance.isComplementData()){
+ }
+ if (parentInstance.getCommandType() == CommandType.REPEAT_RUNNING) {
// update current task id to map
- // repeat running does not generate new sub process instance
processMap = findPreviousTaskProcessMap(parentInstance, parentTask);
- if(processMap!= null){
+ if (processMap != null) {
processMap.setParentTaskInstanceId(parentTask.getId());
updateWorkProcessInstanceMap(processMap);
return processMap;
@@ -944,11 +941,11 @@ public class ProcessService {
Integer preTaskId = 0;
List preTaskList = this.findPreviousTaskListByWorkProcessId(parentProcessInstance.getId());
- for(TaskInstance task : preTaskList){
- if(task.getName().equals(parentTask.getName())){
+ for (TaskInstance task : preTaskList) {
+ if (task.getName().equals(parentTask.getName())) {
preTaskId = task.getId();
ProcessInstanceMap map = findWorkProcessMapByParent(parentProcessInstance.getId(), preTaskId);
- if(map!=null){
+ if (map != null) {
return map;
}
}
@@ -960,66 +957,111 @@ public class ProcessService {
/**
* create sub work process command
+ *
* @param parentProcessInstance parentProcessInstance
- * @param task task
+ * @param task task
*/
- private void createSubWorkProcessCommand(ProcessInstance parentProcessInstance,
- TaskInstance task){
- if(!task.isSubProcess()){
+ public void createSubWorkProcess(ProcessInstance parentProcessInstance, TaskInstance task) {
+ if (!task.isSubProcess()) {
return;
}
- ProcessInstanceMap instanceMap = setProcessInstanceMap(parentProcessInstance, task);
- TaskNode taskNode = JSONUtils.parseObject(task.getTaskJson(), TaskNode.class);
- Map subProcessParam = JSONUtils.toMap(taskNode.getParams());
- Integer childDefineId = Integer.parseInt(subProcessParam.get(Constants.CMDPARAM_SUB_PROCESS_DEFINE_ID));
-
- ProcessInstance childInstance = findSubProcessInstance(parentProcessInstance.getId(), task.getId());
-
- CommandType fatherType = parentProcessInstance.getCommandType();
- CommandType commandType = fatherType;
- if(childInstance == null){
- String fatherHistoryCommand = parentProcessInstance.getHistoryCmd();
- // sub process must begin with schedule/complement data
- // if father begin with scheduler/complement data
- if(fatherHistoryCommand.startsWith(CommandType.SCHEDULER.toString()) ||
- fatherHistoryCommand.startsWith(CommandType.COMPLEMENT_DATA.toString())){
- commandType = CommandType.valueOf(fatherHistoryCommand.split(Constants.COMMA)[0]);
- }
+ //check create sub work flow firstly
+ ProcessInstanceMap instanceMap = findWorkProcessMapByParent(parentProcessInstance.getId(), task.getId());
+ if (null != instanceMap && CommandType.RECOVER_TOLERANCE_FAULT_PROCESS == parentProcessInstance.getCommandType()) {
+ // recover failover tolerance would not create a new command when the sub command already have been created
+ return;
}
-
- if(childInstance != null){
- childInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
- updateProcessInstance(childInstance);
+ instanceMap = setProcessInstanceMap(parentProcessInstance, task);
+ ProcessInstance childInstance = null;
+ if (instanceMap.getProcessInstanceId() != 0) {
+ childInstance = findProcessInstanceById(instanceMap.getProcessInstanceId());
}
+ Command subProcessCommand = createSubProcessCommand(parentProcessInstance, childInstance, instanceMap, task);
+ updateSubProcessDefinitionByParent(parentProcessInstance, subProcessCommand.getProcessDefinitionId());
+ initSubInstanceState(childInstance);
+ createCommand(subProcessCommand);
+ logger.info("sub process command created: {} ", subProcessCommand);
+ }
+
+ /**
+ * complement data needs transform parent parameter to child.
+ * @param instanceMap
+ * @param parentProcessInstance
+ * @return
+ */
+ private String getSubWorkFlowParam(ProcessInstanceMap instanceMap, ProcessInstance parentProcessInstance) {
// set sub work process command
String processMapStr = JSONUtils.toJsonString(instanceMap);
Map cmdParam = JSONUtils.toMap(processMapStr);
-
- if(commandType == CommandType.COMPLEMENT_DATA ||
- (childInstance != null && childInstance.isComplementData())){
+ if (parentProcessInstance.isComplementData()) {
Map parentParam = JSONUtils.toMap(parentProcessInstance.getCommandParam());
- String endTime = parentParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE);
- String startTime = parentParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE);
+ String endTime = parentParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE);
+ String startTime = parentParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE);
cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, endTime);
cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, startTime);
processMapStr = JSONUtils.toJsonString(cmdParam);
}
+ return processMapStr;
+ }
- updateSubProcessDefinitionByParent(parentProcessInstance, childDefineId);
+ /**
+ * create sub work process command
+ * @param parentProcessInstance
+ * @param childInstance
+ * @param instanceMap
+ * @param task
+ */
+ public Command createSubProcessCommand(ProcessInstance parentProcessInstance,
+ ProcessInstance childInstance,
+ ProcessInstanceMap instanceMap,
+ TaskInstance task) {
+ CommandType commandType = getSubCommandType(parentProcessInstance, childInstance);
+ TaskNode taskNode = JSONUtils.parseObject(task.getTaskJson(), TaskNode.class);
+ Map subProcessParam = JSONUtils.toMap(taskNode.getParams());
+ Integer childDefineId = Integer.parseInt(subProcessParam.get(Constants.CMDPARAM_SUB_PROCESS_DEFINE_ID));
+ String processParam = getSubWorkFlowParam(instanceMap, parentProcessInstance);
+
+ return new Command(
+ commandType,
+ TaskDependType.TASK_POST,
+ parentProcessInstance.getFailureStrategy(),
+ parentProcessInstance.getExecutorId(),
+ childDefineId,
+ processParam,
+ parentProcessInstance.getWarningType(),
+ parentProcessInstance.getWarningGroupId(),
+ parentProcessInstance.getScheduleTime(),
+ parentProcessInstance.getProcessInstancePriority()
+ );
+ }
+
+ /**
+ * initialize sub work flow state
+ * child instance state would be initialized when 'recovery from pause/stop/failure'
+ * @param childInstance
+ */
+ private void initSubInstanceState(ProcessInstance childInstance) {
+ if (childInstance != null) {
+ childInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+ updateProcessInstance(childInstance);
+ }
+ }
- Command command = new Command();
- command.setWarningType(parentProcessInstance.getWarningType());
- command.setWarningGroupId(parentProcessInstance.getWarningGroupId());
- command.setFailureStrategy(parentProcessInstance.getFailureStrategy());
- command.setProcessDefinitionId(childDefineId);
- command.setScheduleTime(parentProcessInstance.getScheduleTime());
- command.setExecutorId(parentProcessInstance.getExecutorId());
- command.setCommandParam(processMapStr);
- command.setCommandType(commandType);
- command.setProcessInstancePriority(parentProcessInstance.getProcessInstancePriority());
- command.setWorkerGroup(parentProcessInstance.getWorkerGroup());
- createCommand(command);
- logger.info("sub process command created: {} ", command.toString());
+ /**
+ * get sub work flow command type
+ * child instance exist: child command = fatherCommand
+ * child instance not exists: child command = fatherCommand[0]
+ *
+ * @param parentProcessInstance
+ * @return
+ */
+ private CommandType getSubCommandType(ProcessInstance parentProcessInstance, ProcessInstance childInstance) {
+ CommandType commandType = parentProcessInstance.getCommandType();
+ if (childInstance == null) {
+ String fatherHistoryCommand = parentProcessInstance.getHistoryCmd();
+ commandType = CommandType.valueOf(fatherHistoryCommand.split(Constants.COMMA)[0]);
+ }
+ return commandType;
}
/**
@@ -1030,7 +1072,7 @@ public class ProcessService {
private void updateSubProcessDefinitionByParent(ProcessInstance parentProcessInstance, int childDefinitionId) {
ProcessDefinition fatherDefinition = this.findProcessDefineById(parentProcessInstance.getProcessDefinitionId());
ProcessDefinition childDefinition = this.findProcessDefineById(childDefinitionId);
- if(childDefinition != null && fatherDefinition != null){
+ if (childDefinition != null && fatherDefinition != null) {
childDefinition.setReceivers(fatherDefinition.getReceivers());
childDefinition.setReceiversCc(fatherDefinition.getReceiversCc());
processDefineMapper.updateById(childDefinition);
@@ -1043,22 +1085,22 @@ public class ProcessService {
* @param processInstance processInstance
* @return task instance
*/
- public TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance){
+ public TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance) {
ExecutionStatus processInstanceState = processInstance.getState();
- if(taskInstance.getState().typeIsFailure()){
- if(taskInstance.isSubProcess()){
- taskInstance.setRetryTimes(taskInstance.getRetryTimes() + 1 );
- }else {
+ if (taskInstance.getState().typeIsFailure()) {
+ if (taskInstance.isSubProcess()) {
+ taskInstance.setRetryTimes(taskInstance.getRetryTimes() + 1);
+ } else {
- if( processInstanceState != ExecutionStatus.READY_STOP
- && processInstanceState != ExecutionStatus.READY_PAUSE){
+ if (processInstanceState != ExecutionStatus.READY_STOP
+ && processInstanceState != ExecutionStatus.READY_PAUSE) {
// failure task set invalid
taskInstance.setFlag(Flag.NO);
updateTaskInstance(taskInstance);
// crate new task instance
- if(taskInstance.getState() != ExecutionStatus.NEED_FAULT_TOLERANCE){
- taskInstance.setRetryTimes(taskInstance.getRetryTimes() + 1 );
+ if (taskInstance.getState() != ExecutionStatus.NEED_FAULT_TOLERANCE) {
+ taskInstance.setRetryTimes(taskInstance.getRetryTimes() + 1);
}
taskInstance.setSubmitTime(null);
taskInstance.setStartTime(null);
@@ -1079,13 +1121,12 @@ public class ProcessService {
taskInstance.setFirstSubmitTime(taskInstance.getSubmitTime());
}
boolean saveResult = saveTaskInstance(taskInstance);
- if(!saveResult){
+ if (!saveResult) {
return null;
}
return taskInstance;
}
-
/**
* ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskInstanceId}_${task executed by ip1},${ip2}...
* The tasks with the highest priority are selected by comparing the priorities of the above four levels from high to low.
@@ -1096,7 +1137,7 @@ public class ProcessService {
String taskWorkerGroup = getTaskWorkerGroup(taskInstance);
ProcessInstance processInstance = this.findProcessInstanceById(taskInstance.getProcessInstanceId());
- if(processInstance == null){
+ if (processInstance == null) {
logger.error("process instance is null. please check the task info, task id: " + taskInstance.getId());
return "";
}
@@ -1124,9 +1165,9 @@ public class ProcessService {
* @param processInstanceState processInstanceState
* @return process instance state
*/
- public ExecutionStatus getSubmitTaskState(TaskInstance taskInstance, ExecutionStatus processInstanceState){
+ public ExecutionStatus getSubmitTaskState(TaskInstance taskInstance, ExecutionStatus processInstanceState) {
ExecutionStatus state = taskInstance.getState();
- if(
+ if (
// running, delayed or killed
// the task already exists in task queue
// return state
@@ -1134,17 +1175,17 @@ public class ProcessService {
|| state == ExecutionStatus.DELAY_EXECUTION
|| state == ExecutionStatus.KILL
|| checkTaskExistsInTaskQueue(taskInstance)
- ){
+ ) {
return state;
}
//return pasue /stop if process instance state is ready pause / stop
// or return submit success
- if( processInstanceState == ExecutionStatus.READY_PAUSE){
+ if (processInstanceState == ExecutionStatus.READY_PAUSE) {
state = ExecutionStatus.PAUSE;
- }else if(processInstanceState == ExecutionStatus.READY_STOP
+ } else if (processInstanceState == ExecutionStatus.READY_STOP
|| !checkProcessStrategy(taskInstance)) {
state = ExecutionStatus.KILL;
- }else{
+ } else {
state = ExecutionStatus.SUBMITTED_SUCCESS;
}
return state;
@@ -1155,16 +1196,16 @@ public class ProcessService {
* @param taskInstance taskInstance
* @return check strategy result
*/
- private boolean checkProcessStrategy(TaskInstance taskInstance){
+ private boolean checkProcessStrategy(TaskInstance taskInstance) {
ProcessInstance processInstance = this.findProcessInstanceById(taskInstance.getProcessInstanceId());
FailureStrategy failureStrategy = processInstance.getFailureStrategy();
- if(failureStrategy == FailureStrategy.CONTINUE){
+ if (failureStrategy == FailureStrategy.CONTINUE) {
return true;
}
List taskInstances = this.findValidTaskListByProcessId(taskInstance.getProcessInstanceId());
- for(TaskInstance task : taskInstances){
- if(task.getState() == ExecutionStatus.FAILURE){
+ for (TaskInstance task : taskInstances) {
+ if (task.getState() == ExecutionStatus.FAILURE) {
return false;
}
}
@@ -1176,8 +1217,8 @@ public class ProcessService {
* @param taskInstance taskInstance
* @return whether taskinstance exists queue
*/
- public boolean checkTaskExistsInTaskQueue(TaskInstance taskInstance){
- if(taskInstance.isSubProcess()){
+ public boolean checkTaskExistsInTaskQueue(TaskInstance taskInstance) {
+ if (taskInstance.isSubProcess()) {
return false;
}
@@ -1190,9 +1231,9 @@ public class ProcessService {
* create a new process instance
* @param processInstance processInstance
*/
- public void createProcessInstance(ProcessInstance processInstance){
+ public void createProcessInstance(ProcessInstance processInstance) {
- if (processInstance != null){
+ if (processInstance != null) {
processInstanceMapper.insert(processInstance);
}
}
@@ -1201,15 +1242,15 @@ public class ProcessService {
* insert or update work process instance to data base
* @param processInstance processInstance
*/
- public void saveProcessInstance(ProcessInstance processInstance){
+ public void saveProcessInstance(ProcessInstance processInstance) {
- if (processInstance == null){
+ if (processInstance == null) {
logger.error("save error, process instance is null!");
- return ;
+ return;
}
- if(processInstance.getId() != 0){
+ if (processInstance.getId() != 0) {
processInstanceMapper.updateById(processInstance);
- }else{
+ } else {
createProcessInstance(processInstance);
}
}
@@ -1219,10 +1260,10 @@ public class ProcessService {
* @param command command
* @return save command result
*/
- public int saveCommand(Command command){
- if(command.getId() != 0){
+ public int saveCommand(Command command) {
+ if (command.getId() != 0) {
return commandMapper.updateById(command);
- }else{
+ } else {
return commandMapper.insert(command);
}
}
@@ -1232,10 +1273,10 @@ public class ProcessService {
* @param taskInstance taskInstance
* @return save task instance result
*/
- public boolean saveTaskInstance(TaskInstance taskInstance){
- if(taskInstance.getId() != 0){
+ public boolean saveTaskInstance(TaskInstance taskInstance) {
+ if (taskInstance.getId() != 0) {
return updateTaskInstance(taskInstance);
- }else{
+ } else {
return createTaskInstance(taskInstance);
}
}
@@ -1255,15 +1296,16 @@ public class ProcessService {
* @param taskInstance taskInstance
* @return update task instance result
*/
- public boolean updateTaskInstance(TaskInstance taskInstance){
+ public boolean updateTaskInstance(TaskInstance taskInstance) {
int count = taskInstanceMapper.updateById(taskInstance);
return count > 0;
}
+
/**
* delete a command by id
* @param id id
*/
- public void delCommandByid(int id) {
+ public void delCommandById(int id) {
commandMapper.deleteById(id);
}
@@ -1272,20 +1314,19 @@ public class ProcessService {
* @param taskId task id
* @return task intance
*/
- public TaskInstance findTaskInstanceById(Integer taskId){
+ public TaskInstance findTaskInstanceById(Integer taskId) {
return taskInstanceMapper.selectById(taskId);
}
-
/**
* package task instance,associate processInstance and processDefine
* @param taskInstId taskInstId
* @return task instance
*/
- public TaskInstance getTaskInstanceDetailByTaskId(int taskInstId){
+ public TaskInstance getTaskInstanceDetailByTaskId(int taskInstId) {
// get task instance
TaskInstance taskInstance = findTaskInstanceById(taskInstId);
- if(taskInstance == null){
+ if (taskInstance == null) {
return taskInstance;
}
// get process instance
@@ -1298,14 +1339,13 @@ public class ProcessService {
return taskInstance;
}
-
/**
* get id list by task state
* @param instanceId instanceId
* @param state state
* @return task instance states
*/
- public List findTaskIdByInstanceState(int instanceId, ExecutionStatus state){
+ public List findTaskIdByInstanceState(int instanceId, ExecutionStatus state) {
return taskInstanceMapper.queryTaskByProcessIdAndState(instanceId, state.ordinal());
}
@@ -1314,8 +1354,8 @@ public class ProcessService {
* @param processInstanceId processInstanceId
* @return task instance list
*/
- public List findValidTaskListByProcessId(Integer processInstanceId){
- return taskInstanceMapper.findValidTaskListByProcessId(processInstanceId, Flag.YES);
+ public List findValidTaskListByProcessId(Integer processInstanceId) {
+ return taskInstanceMapper.findValidTaskListByProcessId(processInstanceId, Flag.YES);
}
/**
@@ -1323,7 +1363,7 @@ public class ProcessService {
* @param processInstanceId processInstanceId
* @return task instance list
*/
- public List findPreviousTaskListByWorkProcessId(Integer processInstanceId){
+ public List findPreviousTaskListByWorkProcessId(Integer processInstanceId) {
return taskInstanceMapper.findValidTaskListByProcessId(processInstanceId, Flag.NO);
}
@@ -1332,19 +1372,18 @@ public class ProcessService {
* @param processInstanceMap processInstanceMap
* @return update process instance result
*/
- public int updateWorkProcessInstanceMap(ProcessInstanceMap processInstanceMap){
+ public int updateWorkProcessInstanceMap(ProcessInstanceMap processInstanceMap) {
return processInstanceMapMapper.updateById(processInstanceMap);
}
-
/**
* create work process instance map
* @param processInstanceMap processInstanceMap
* @return create process instance result
*/
- public int createWorkProcessInstanceMap(ProcessInstanceMap processInstanceMap){
+ public int createWorkProcessInstanceMap(ProcessInstanceMap processInstanceMap) {
Integer count = 0;
- if(processInstanceMap !=null){
+ if (processInstanceMap != null) {
return processInstanceMapMapper.insert(processInstanceMap);
}
return count;
@@ -1356,7 +1395,7 @@ public class ProcessService {
* @param parentTaskId parentTaskId
* @return process instance map
*/
- public ProcessInstanceMap findWorkProcessMapByParent(Integer parentWorkProcessId, Integer parentTaskId){
+ public ProcessInstanceMap findWorkProcessMapByParent(Integer parentWorkProcessId, Integer parentTaskId) {
return processInstanceMapMapper.queryByParentId(parentWorkProcessId, parentTaskId);
}
@@ -1365,7 +1404,7 @@ public class ProcessService {
* @param parentWorkProcessId parentWorkProcessId
* @return delete process map result
*/
- public int deleteWorkProcessMapByParentId(int parentWorkProcessId){
+ public int deleteWorkProcessMapByParentId(int parentWorkProcessId) {
return processInstanceMapMapper.deleteByParentProcessId(parentWorkProcessId);
}
@@ -1376,10 +1415,10 @@ public class ProcessService {
* @param parentTaskId parentTaskId
* @return process instance
*/
- public ProcessInstance findSubProcessInstance(Integer parentProcessId, Integer parentTaskId){
+ public ProcessInstance findSubProcessInstance(Integer parentProcessId, Integer parentTaskId) {
ProcessInstance processInstance = null;
ProcessInstanceMap processInstanceMap = processInstanceMapMapper.queryByParentId(parentProcessId, parentTaskId);
- if(processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0){
+ if (processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0) {
return processInstance;
}
processInstance = findProcessInstanceById(processInstanceMap.getProcessInstanceId());
@@ -1394,14 +1433,13 @@ public class ProcessService {
public ProcessInstance findParentProcessInstance(Integer subProcessId) {
ProcessInstance processInstance = null;
ProcessInstanceMap processInstanceMap = processInstanceMapMapper.queryBySubProcessId(subProcessId);
- if(processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0){
+ if (processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0) {
return processInstance;
}
processInstance = findProcessInstanceById(processInstanceMap.getParentProcessInstanceId());
return processInstance;
}
-
/**
* change task state
* @param state state
@@ -1429,7 +1467,7 @@ public class ProcessService {
* @param processInstance processInstance
* @return update process instance result
*/
- public int updateProcessInstance(ProcessInstance processInstance){
+ public int updateProcessInstance(ProcessInstance processInstance) {
return processInstanceMapper.updateById(processInstance);
}
@@ -1446,9 +1484,9 @@ public class ProcessService {
*/
public int updateProcessInstance(Integer processInstanceId, String processJson,
String globalParams, Date scheduleTime, Flag flag,
- String locations, String connects){
+ String locations, String connects) {
ProcessInstance processInstance = processInstanceMapper.queryDetailById(processInstanceId);
- if(processInstance!= null){
+ if (processInstance != null) {
processInstance.setProcessInstanceJson(processJson);
processInstance.setGlobalParams(globalParams);
processInstance.setScheduleTime(scheduleTime);
@@ -1464,17 +1502,20 @@ public class ProcessService {
* @param state state
* @param endTime endTime
* @param taskInstId taskInstId
+ * @param varPool varPool
*/
public void changeTaskState(ExecutionStatus state,
Date endTime,
int processId,
String appIds,
- int taskInstId) {
+ int taskInstId,
+ String varPool) {
TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId);
taskInstance.setPid(processId);
taskInstance.setAppLink(appIds);
taskInstance.setState(state);
taskInstance.setEndTime(endTime);
+ taskInstance.setVarPool(varPool);
saveTaskInstance(taskInstance);
}
@@ -1483,31 +1524,17 @@ public class ProcessService {
* @param intList intList
* @return string list
*/
- public List convertIntListToString(List intList){
- if(intList == null){
+ public List convertIntListToString(List intList) {
+ if (intList == null) {
return new ArrayList<>();
}
List result = new ArrayList(intList.size());
- for(Integer intVar : intList){
+ for (Integer intVar : intList) {
result.add(String.valueOf(intVar));
}
return result;
}
- /**
- * update pid and app links field by task instance id
- * @param taskInstId taskInstId
- * @param pid pid
- * @param appLinks appLinks
- */
- public void updatePidByTaskInstId(int taskInstId, int pid,String appLinks) {
-
- TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId);
- taskInstance.setPid(pid);
- taskInstance.setAppLink(appLinks);
- saveTaskInstance(taskInstance);
- }
-
/**
* query schedule by id
* @param id id
@@ -1531,8 +1558,7 @@ public class ProcessService {
* @param host host
* @return process instance list
*/
- public List queryNeedFailoverProcessInstances(String host){
-
+ public List queryNeedFailoverProcessInstances(String host) {
return processInstanceMapper.queryByHostAndStatus(host, stateArray);
}
@@ -1541,7 +1567,7 @@ public class ProcessService {
* @param processInstance processInstance
*/
@Transactional(rollbackFor = RuntimeException.class)
- public void processNeedFailoverProcessInstances(ProcessInstance processInstance){
+ public void processNeedFailoverProcessInstances(ProcessInstance processInstance) {
//1 update processInstance host is null
processInstance.setHost(Constants.NULL);
processInstanceMapper.updateById(processInstance);
@@ -1560,7 +1586,7 @@ public class ProcessService {
* @param host host
* @return task instance list
*/
- public List queryNeedFailoverTaskInstances(String host){
+ public List queryNeedFailoverTaskInstances(String host) {
return taskInstanceMapper.queryByHostAndStatus(host,
stateArray);
}
@@ -1570,11 +1596,10 @@ public class ProcessService {
* @param id id
* @return datasource
*/
- public DataSource findDataSourceById(int id){
+ public DataSource findDataSourceById(int id) {
return dataSourceMapper.selectById(id);
}
-
/**
* update process instance state by id
* @param processInstanceId processInstanceId
@@ -1593,9 +1618,9 @@ public class ProcessService {
* @param taskId taskId
* @return process instance
*/
- public ProcessInstance findProcessInstanceByTaskId(int taskId){
+ public ProcessInstance findProcessInstanceByTaskId(int taskId) {
TaskInstance taskInstance = taskInstanceMapper.selectById(taskId);
- if(taskInstance!= null){
+ if (taskInstance != null) {
return processInstanceMapper.selectById(taskInstance.getProcessInstanceId());
}
return null;
@@ -1606,7 +1631,7 @@ public class ProcessService {
* @param ids ids
* @return udf function list
*/
- public List queryUdfFunListByids(int[] ids){
+ public List queryUdfFunListByIds(int[] ids) {
return udfFuncMapper.queryUdfByIdStr(ids, null);
}
@@ -1616,7 +1641,7 @@ public class ProcessService {
* @param resourceType resource type
* @return tenant code
*/
- public String queryTenantCodeByResName(String resName,ResourceType resourceType){
+ public String queryTenantCodeByResName(String resName,ResourceType resourceType) {
return resourceMapper.queryTenantCodeByResourceName(resName, resourceType.ordinal());
}
@@ -1625,7 +1650,7 @@ public class ProcessService {
* @param ids ids
* @return schedule list
*/
- public List selectAllByProcessDefineId(int[] ids){
+ public List selectAllByProcessDefineId(int[] ids) {
return scheduleMapper.selectAllByProcessDefineArray(
ids);
}
@@ -1640,7 +1665,7 @@ public class ProcessService {
*/
public CycleDependency getCycleDependency(int masterId, int processDefinitionId, Date scheduledFireTime) throws Exception {
List list = getCycleDependencies(masterId,new int[]{processDefinitionId},scheduledFireTime);
- return list.size()>0 ? list.get(0) : null;
+ return list.size() > 0 ? list.get(0) : null;
}
@@ -1658,29 +1683,28 @@ public class ProcessService {
logger.warn("ids[] is empty!is invalid!");
return cycleDependencyList;
}
- if(scheduledFireTime == null){
+ if (scheduledFireTime == null) {
logger.warn("scheduledFireTime is null!is invalid!");
return cycleDependencyList;
}
-
String strCrontab = "";
CronExpression depCronExpression;
Cron depCron;
List list;
List schedules = this.selectAllByProcessDefineId(ids);
// for all scheduling information
- for(Schedule depSchedule:schedules){
+ for (Schedule depSchedule:schedules) {
strCrontab = depSchedule.getCrontab();
depCronExpression = CronUtils.parse2CronExpression(strCrontab);
depCron = CronUtils.parse2Cron(strCrontab);
CycleEnum cycleEnum = CronUtils.getMiniCycle(depCron);
- if(cycleEnum == null){
+ if (cycleEnum == null) {
logger.error("{} is not valid",strCrontab);
continue;
}
Calendar calendar = Calendar.getInstance();
- switch (cycleEnum){
+ switch (cycleEnum) {
/*case MINUTE:
calendar.add(Calendar.MINUTE,-61);*/
case HOUR:
@@ -1701,13 +1725,13 @@ public class ProcessService {
}
Date start = calendar.getTime();
- if(depSchedule.getProcessDefinitionId() == masterId){
+ if (depSchedule.getProcessDefinitionId() == masterId) {
list = CronUtils.getSelfFireDateList(start, scheduledFireTime, depCronExpression);
- }else {
+ } else {
list = CronUtils.getFireDateList(start, scheduledFireTime, depCronExpression);
}
- if(list.size()>=1){
- start = list.get(list.size()-1);
+ if (list.size() >= 1) {
+ start = list.get(list.size() - 1);
CycleDependency dependency = new CycleDependency(depSchedule.getProcessDefinitionId(),start, CronUtils.getExpirationTime(start, cycleEnum), cycleEnum);
cycleDependencyList.add(dependency);
}
@@ -1759,22 +1783,20 @@ public class ProcessService {
* @param processInstanceId processInstanceId
* @return queue
*/
- public String queryUserQueueByProcessInstanceId(int processInstanceId){
+ public String queryUserQueueByProcessInstanceId(int processInstanceId) {
String queue = "";
ProcessInstance processInstance = processInstanceMapper.selectById(processInstanceId);
- if(processInstance == null){
+ if (processInstance == null) {
return queue;
}
User executor = userMapper.selectById(processInstance.getExecutorId());
- if(executor != null){
+ if (executor != null) {
queue = executor.getQueue();
}
return queue;
}
-
-
/**
* get task worker group
* @param taskInstance taskInstance
@@ -1783,13 +1805,13 @@ public class ProcessService {
public String getTaskWorkerGroup(TaskInstance taskInstance) {
String workerGroup = taskInstance.getWorkerGroup();
- if(StringUtils.isNotBlank(workerGroup)){
+ if (StringUtils.isNotBlank(workerGroup)) {
return workerGroup;
}
int processInstanceId = taskInstance.getProcessInstanceId();
ProcessInstance processInstance = findProcessInstanceById(processInstanceId);
- if(processInstance != null){
+ if (processInstance != null) {
return processInstance.getWorkerGroup();
}
logger.info("task : {} will use default worker group", taskInstance.getId());
@@ -1801,15 +1823,15 @@ public class ProcessService {
* @param userId userId
* @return project list
*/
- public List getProjectListHavePerm(int userId){
+ public List getProjectListHavePerm(int userId) {
List createProjects = projectMapper.queryProjectCreatedByUser(userId);
List authedProjects = projectMapper.queryAuthedProjectListByUserId(userId);
- if(createProjects == null){
+ if (createProjects == null) {
createProjects = new ArrayList<>();
}
- if(authedProjects != null){
+ if (authedProjects != null) {
createProjects.addAll(authedProjects);
}
return createProjects;
@@ -1820,10 +1842,10 @@ public class ProcessService {
* @param userId userId
* @return project ids
*/
- public List getProjectIdListHavePerm(int userId){
+ public List getProjectIdListHavePerm(int userId) {
List projectIdList = new ArrayList<>();
- for(Project project : getProjectListHavePerm(userId)){
+ for (Project project : getProjectListHavePerm(userId)) {
projectIdList.add(project.getId());
}
return projectIdList;
@@ -1835,13 +1857,13 @@ public class ProcessService {
* @param needChecks data source id array
* @return unauthorized udf function list
*/
- public List listUnauthorized(int userId,T[] needChecks,AuthorizationType authorizationType){
+ public List listUnauthorized(int userId,T[] needChecks,AuthorizationType authorizationType) {
List resultList = new ArrayList();
if (Objects.nonNull(needChecks) && needChecks.length > 0) {
Set originResSet = new HashSet(Arrays.asList(needChecks));
- switch (authorizationType){
+ switch (authorizationType) {
case RESOURCE_FILE_ID:
Set authorizedResourceFiles = resourceMapper.listAuthorizedResourceById(userId, needChecks).stream().map(t -> t.getId()).collect(toSet());
originResSet.removeAll(authorizedResourceFiles);
@@ -1862,6 +1884,8 @@ public class ProcessService {
Set authorizedUdfs = udfFuncMapper.listAuthorizedUdfFunc(userId, needChecks).stream().map(t -> t.getId()).collect(toSet());
originResSet.removeAll(authorizedUdfs);
break;
+ default:
+ break;
}
resultList.addAll(originResSet);
@@ -1875,7 +1899,7 @@ public class ProcessService {
* @param userId user id
* @return User
*/
- public User getUserById(int userId){
+ public User getUserById(int userId) {
return userMapper.selectById(userId);
}
@@ -1884,17 +1908,16 @@ public class ProcessService {
* @param resoruceId resource id
* @return Resource
*/
- public Resource getResourceById(int resoruceId){
+ public Resource getResourceById(int resoruceId) {
return resourceMapper.selectById(resoruceId);
}
-
/**
* list resources by ids
* @param resIds resIds
* @return resource list
*/
- public List listResourceByIds(Integer[] resIds){
+ public List listResourceByIds(Integer[] resIds) {
return resourceMapper.listResourceByIds(resIds);
}
@@ -1903,11 +1926,11 @@ public class ProcessService {
* @param taskInstance
* @return
*/
- public String formatTaskAppId(TaskInstance taskInstance){
+ public String formatTaskAppId(TaskInstance taskInstance) {
ProcessDefinition definition = this.findProcessDefineById(taskInstance.getProcessDefinitionId());
ProcessInstance processInstanceById = this.findProcessInstanceById(taskInstance.getProcessInstanceId());
- if(definition == null || processInstanceById == null){
+ if (definition == null || processInstanceById == null) {
return "";
}
return String.format("%s_%s_%s",
diff --git a/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java
new file mode 100644
index 0000000000..74b52bb316
--- /dev/null
+++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.service.process;
+
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.enums.WarningType;
+import org.apache.dolphinscheduler.common.utils.DateUtils;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.Command;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstanceMap;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * process service test
+ */
+public class ProcessServiceTest {
+
+ @Test
+ public void testCreateSubCommand() {
+ ProcessService processService = new ProcessService();
+ ProcessInstance parentInstance = new ProcessInstance();
+ parentInstance.setProcessDefinitionId(1);
+ parentInstance.setWarningType(WarningType.SUCCESS);
+ parentInstance.setWarningGroupId(0);
+
+ TaskInstance task = new TaskInstance();
+ task.setTaskJson("{\"params\":{\"processDefinitionId\":100}}");
+ task.setId(10);
+
+ ProcessInstance childInstance = null;
+ ProcessInstanceMap instanceMap = new ProcessInstanceMap();
+ instanceMap.setParentProcessInstanceId(1);
+ instanceMap.setParentTaskInstanceId(10);
+ Command command = null;
+
+ //father history: start; child null == command type: start
+ parentInstance.setHistoryCmd("START_PROCESS");
+ parentInstance.setCommandType(CommandType.START_PROCESS);
+ command = processService.createSubProcessCommand(
+ parentInstance, childInstance, instanceMap, task
+ );
+ Assert.assertEquals(CommandType.START_PROCESS, command.getCommandType());
+
+ //father history: start,start failure; child null == command type: start
+ parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
+ parentInstance.setHistoryCmd("START_PROCESS,START_FAILURE_TASK_PROCESS");
+ command = processService.createSubProcessCommand(
+ parentInstance, childInstance, instanceMap, task
+ );
+ Assert.assertEquals(CommandType.START_PROCESS, command.getCommandType());
+
+ //father history: scheduler,start failure; child null == command type: scheduler
+ parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
+ parentInstance.setHistoryCmd("SCHEDULER,START_FAILURE_TASK_PROCESS");
+ command = processService.createSubProcessCommand(
+ parentInstance, childInstance, instanceMap, task
+ );
+ Assert.assertEquals(CommandType.SCHEDULER, command.getCommandType());
+
+ //father history: complement,start failure; child null == command type: complement
+
+ String startString = "2020-01-01 00:00:00";
+ String endString = "2020-01-10 00:00:00";
+ parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
+ parentInstance.setHistoryCmd("COMPLEMENT_DATA,START_FAILURE_TASK_PROCESS");
+ Map complementMap = new HashMap<>();
+ complementMap.put(Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE, startString);
+ complementMap.put(Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE, endString);
+ parentInstance.setCommandParam(JSONUtils.toJsonString(complementMap));
+ command = processService.createSubProcessCommand(
+ parentInstance, childInstance, instanceMap, task
+ );
+ Assert.assertEquals(CommandType.COMPLEMENT_DATA, command.getCommandType());
+
+ JsonNode complementDate = JSONUtils.parseObject(command.getCommandParam());
+ Date start = DateUtils.stringToDate(complementDate.get(Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE).asText());
+ Date end = DateUtils.stringToDate(complementDate.get(Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE).asText());
+ Assert.assertEquals(startString, DateUtils.dateToString(start));
+ Assert.assertEquals(endString, DateUtils.dateToString(end));
+
+ //father history: start,failure,start failure; child not null == command type: start failure
+ childInstance = new ProcessInstance();
+ parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
+ parentInstance.setHistoryCmd("START_PROCESS,START_FAILURE_TASK_PROCESS");
+ command = processService.createSubProcessCommand(
+ parentInstance, childInstance, instanceMap, task
+ );
+ Assert.assertEquals(CommandType.START_FAILURE_TASK_PROCESS, command.getCommandType());
+ }
+}
diff --git a/dolphinscheduler-ui/package.json b/dolphinscheduler-ui/package.json
index a5642c8f97..9624fa6212 100644
--- a/dolphinscheduler-ui/package.json
+++ b/dolphinscheduler-ui/package.json
@@ -20,6 +20,7 @@
"clipboard": "^2.0.1",
"codemirror": "^5.43.0",
"d3": "^3.5.17",
+ "dagre": "^0.8.5",
"dayjs": "^1.7.8",
"echarts": "4.1.0",
"html2canvas": "^0.5.0-beta4",
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js
index 2e60929577..18fbd94341 100755
--- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js
@@ -236,6 +236,13 @@ const tasksState = {
color: '#5102ce',
icoUnicode: 'ans-icon-coin',
isSpin: false
+ },
+ FORCED_SUCCESS: {
+ id: 13,
+ desc: `${i18n.$t('Forced success')}`,
+ color: '#5102ce',
+ icoUnicode: 'ans-icon-success-solid',
+ isSpin: false
}
}
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js
index ff8a4528d5..74671cca57 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js
@@ -22,6 +22,7 @@ import { jsPlumb } from 'jsplumb'
import JSP from './plugIn/jsPlumbHandle'
import DownChart from './plugIn/downChart'
import store from '@/conf/home/store'
+import dagre from "dagre"
/**
* Prototype method
@@ -115,202 +116,40 @@ Dag.prototype.toolbarEvent = function ({ item, code, is }) {
*/
Dag.prototype.backfill = function (arg) {
if (arg) {
- let locationsValue = store.state.dag.locations
- const locationsValue1 = store.state.dag.locations
- const locationsValue2 = store.state.dag.locations
- const arr = []
- for (const i in locationsValue1) {
- const objs = {}
- objs.id = i
- arr.push(Object.assign(objs, locationsValue1[i])) // Attributes
- }
- const tmp = []
- for (const i in locationsValue2) {
- if (locationsValue2[i].targetarr !== '' && locationsValue2[i].targetarr.split(',').length > 1) {
- tmp.push(locationsValue2[i])
- }
- }
-
- const copy = function (array) {
- const newArray = []
- for (const item of array) {
- newArray.push(item)
- }
- return newArray
- }
-
- const newArr = copy(arr)
- const getNewArr = function () {
- for (let i = 0; i < newArr.length; i++) {
- if (newArr[i].targetarr !== '' && newArr[i].targetarr.split(',').length > 1) {
- newArr[i].targetarr = newArr[i].targetarr.split(',').shift()
- }
- }
- return newArr
- }
- getNewArr()
- /**
- * @description Transform flat data into a tree structure
- * @param {Array} arr Flat data
- * @param {String} pidStr targetarr key name
- * @param {String} idStr id key name
- * @param {String} childrenStr children key name
- */
- const fommat = function ({ arrayList, pidStr = 'targetarr', idStr = 'id', childrenStr = 'children' }) {
- const listOjb = {} // Used to store objects of the form {key: obj}
- const treeList = [] // An array to store the final tree structure data
- // Transform the data into {key: obj} format, which is convenient for the following data processing
- for (let i = 0; i < arrayList.length; i++) {
- listOjb[arrayList[i][idStr]] = arrayList[i]
- }
- // Format data based on pid
- for (let j = 0; j < arrayList.length; j++) {
- // Determine if the parent exists
- // let haveParent = arrayList[j].targetarr.split(',').length>1?listOjb[arrayList[j].targetarr.split(',')[0]]:listOjb[arrayList[j][pidStr]]
- const haveParent = listOjb[arrayList[j][pidStr]]
- if (haveParent) {
- // If there is no parent children field, create a children field
- !haveParent[childrenStr] && (haveParent[childrenStr] = [])
- // Insert child in parent
- haveParent[childrenStr].push(arrayList[j])
- } else {
- // If there is no parent, insert directly into the outermost layer
- treeList.push(arrayList[j])
- }
- }
- return treeList
- }
- const datas = fommat({ arrayList: newArr, pidStr: 'targetarr' })
- // Count the number of leaf nodes
- const getLeafCountTree = function (json) {
- if (!json.children) {
- json.colspan = 1
- return 1
- } else {
- let leafCount = 0
- for (let i = 0; i < json.children.length; i++) {
- leafCount = leafCount + getLeafCountTree(json.children[i])
- }
- json.colspan = leafCount
- return leafCount
- }
- }
- // Number of tree node levels
- const countTree = getLeafCountTree(datas[0])
- const getMaxFloor = function (treeData) {
- let max = 0
- function each (data, floor) {
- data.forEach(e => {
- e.floor = floor
- e.x = floor * 170
- if (floor > max) {
- max = floor
- }
- if (e.children) {
- each(e.children, floor + 1)
- }
- })
- }
- each(treeData, 1)
- return max
- }
- getMaxFloor(datas)
- // The last child of each node
- let lastchildren = []
- const forxh = function (list) {
- for (let i = 0; i < list.length; i++) {
- const chlist = list[i]
- if (chlist.children) {
- forxh(chlist.children)
- } else {
- lastchildren.push(chlist)
- }
- }
- }
- forxh(datas)
- // Get all parent nodes above the leaf node
- const treeFindPath = function (tree, func, path, n) {
- if (!tree) return []
- for (const data of tree) {
- path.push(data.name)
- if (func(data)) return path
- if (data.children) {
- const findChildren = treeFindPath(data.children, func, path, n)
- if (findChildren.length) return findChildren
- }
- path.pop()
- }
- return []
- }
- const toLine = function (data) {
- return data.reduce((arrData, { id, name, targetarr, x, y, children = [] }) =>
- arrData.concat([{ id, name, targetarr, x, y }], toLine(children)), [])
- }
- const listarr = toLine(datas)
- const listarrs = toLine(datas)
- const dataObject = {}
- for (let i = 0; i < listarrs.length; i++) {
- delete (listarrs[i].id)
- }
+ const marginX = 100
+ const g = new dagre.graphlib.Graph()
+ g.setGraph({})
+ g.setDefaultEdgeLabel(function () { return {} })
- for (let a = 0; a < listarr.length; a++) {
- dataObject[listarr[a].id] = listarrs[a]
+ for (const i in store.state.dag.locations) {
+ const location = store.state.dag.locations[i]
+ g.setNode(i, { label: i, width: Math.min(location.name.length * 7, 170), height: 150 })
}
- // Comparison function
- const createComparisonFunction = function (propertyName) {
- return function (object1, object2) {
- const value1 = object1[propertyName]
- const value2 = object2[propertyName]
- if (value1 < value2) {
- return -1
- } else if (value1 > value2) {
- return 1
- } else {
- return 0
- }
- }
+ for (const i in store.state.dag.connects) {
+ const connect = store.state.dag.connects[i]
+ g.setEdge(connect['endPointSourceId'], connect['endPointTargetId'])
}
+ dagre.layout(g)
- lastchildren = lastchildren.sort(createComparisonFunction('x'))
-
- // Coordinate value of each leaf node
- for (let a = 0; a < lastchildren.length; a++) {
- dataObject[lastchildren[a].id].y = (a + 1) * 120
- }
- for (let i = 0; i < lastchildren.length; i++) {
- const node = treeFindPath(datas, data => data.targetarr === lastchildren[i].targetarr, [], i + 1)
- for (let j = 0; j < node.length; j++) {
- for (let k = 0; k < listarrs.length; k++) {
- if (node[j] === listarrs[k].name) {
- listarrs[k].y = (i + 1) * 120
- }
- }
- }
- }
- for (let i = 0; i < tmp.length; i++) {
- for (const objs in dataObject) {
- if (tmp[i].name === dataObject[objs].name) {
- dataObject[objs].targetarr = tmp[i].targetarr
- }
- }
- }
- for (let a = 0; a < lastchildren.length; a++) {
- dataObject[lastchildren[a].id].y = (a + 1) * 120
- }
- if (countTree > 1) {
- dataObject[Object.keys(locationsValue1)[0]].y = (countTree / 2) * 120 + 50
- }
-
- locationsValue = dataObject
- const self = this
+ const dataObject = {}
+ g.nodes().forEach(function (v) {
+ const node = g.node(v)
+ const location = store.state.dag.locations[node.label]
+ const obj = {}
+ obj.name = location.name
+ obj.x = node.x + marginX
+ obj.y = node.y
+ obj.targetarr = location.targetarr
+ dataObject[node.label] = obj
+ })
jsPlumb.ready(() => {
JSP.init({
dag: this.dag,
instance: this.instance,
options: {
onRemoveNodes ($id) {
- self.dag.removeEventModelById($id)
+ this.dag.removeEventModelById($id)
}
}
})
@@ -319,20 +158,19 @@ Dag.prototype.backfill = function (arg) {
// connects
connects: _.cloneDeep(store.state.dag.connects),
// Node location information
- locations: _.cloneDeep(locationsValue),
+ locations: _.cloneDeep(dataObject),
// Node data
largeJson: _.cloneDeep(store.state.dag.tasks)
})
})
} else {
- const self = this
jsPlumb.ready(() => {
JSP.init({
dag: this.dag,
instance: this.instance,
options: {
onRemoveNodes ($id) {
- self.dag.removeEventModelById($id)
+ this.dag.removeEventModelById($id)
}
}
})
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/_source/dependentTimeout.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/_source/dependentTimeout.vue
new file mode 100644
index 0000000000..2b2ed78ccc
--- /dev/null
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/_source/dependentTimeout.vue
@@ -0,0 +1,218 @@
+/*
+ * 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.
+ */
+
+
+
+
+ {{$t('Timeout alarm')}}
+
+
+
+
+
+ {{$t('Waiting Dependent start')}}
+
+
+
+
+
+
+
+
+
+
+ {{$t('Waiting Dependent complete')}}
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue
index 8444863aea..b6c48fe5e8 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue
@@ -162,10 +162,18 @@
+
+
+
+
+
@@ -309,7 +323,9 @@
import mSubProcess from './tasks/sub_process'
import mSelectInput from './_source/selectInput'
import mTimeoutAlarm from './_source/timeoutAlarm'
+ import mDependentTimeout from './_source/dependentTimeout'
import mWorkerGroups from './_source/workerGroups'
+ import mPreTasks from './tasks/pre_tasks'
import clickoutside from '@/module/util/clickoutside'
import disabledState from '@/module/mixin/disabledState'
import { isNameExDag, rtBantpl } from './../plugIn/util'
@@ -356,6 +372,8 @@
delayTime: '0',
// Task timeout alarm
timeout: {},
+ // (For Dependent nodes) Wait start timeout alarm
+ waitStartTimeout: {},
// Task priority
taskInstancePriority: 'MEDIUM',
// worker group id
@@ -369,7 +387,11 @@
value: 'failed',
label: `${i18n.$t('failed')}`
}
- ]
+ ],
+ // preTasks
+ preTaskIdsInWorkflow: [],
+ preTasksToAdd: [], // pre-taskIds to add, used in jsplumb connects
+ preTasksToDelete: [], // pre-taskIds to delete, used in jsplumb connects
}
},
/**
@@ -393,6 +415,14 @@
_onDependent (o) {
this.dependence = Object.assign(this.dependence, {}, o)
},
+ /**
+ * Pre-tasks in workflow
+ */
+ _onPreTasks (o) {
+ this.preTaskIdsInWorkflow = o.preTasks
+ this.preTasksToAdd = o.preTasksToAdd
+ this.preTasksToDelete = o.preTasksToDelete
+ },
/**
* cache dependent
*/
@@ -405,6 +435,13 @@
_onTimeout (o) {
this.timeout = Object.assign(this.timeout, {}, o)
},
+ /**
+ * Dependent timeout alarm
+ */
+ _onDependentTimeout (o) {
+ this.timeout = Object.assign(this.timeout, {}, o.waitCompleteTimeout)
+ this.waitStartTimeout = Object.assign(this.waitStartTimeout, {}, o.waitStartTimeout)
+ },
/**
* Click external to close the current component
*/
@@ -483,6 +520,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
+ waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
status: this.status,
@@ -536,13 +574,57 @@
return
}
// Verify task alarm parameters
- if (!this.$refs['timeout']._verification()) {
- return
+ if (this.taskType === 'DEPENDENT') {
+ if (!this.$refs['dependentTimeout']._verification()) {
+ return
+ }
+ } else {
+ if (!this.$refs['timeout']._verification()) {
+ return
+ }
}
+
// Verify node parameters
if (!this.$refs[this.taskType]._verification()) {
return
}
+ // Verify preTasks and update dag-things
+ if (this.$refs['PRE_TASK']) {
+ if (!this.$refs['PRE_TASK']._verification()) {
+ return
+ }
+ else {
+ // Sync data-targetarr
+ $(`#${this.id}`).attr(
+ 'data-targetarr', this.preTaskIdsInWorkflow ? this.preTaskIdsInWorkflow.join(',') : '')
+
+ // Update JSP connections
+ let plumbIns = JSP.JspInstance
+ var targetId = this.id
+
+ // Update new connections
+ this.preTasksToAdd.map(sourceId => {
+ plumbIns.connect({
+ source: sourceId,
+ target: targetId,
+ type: 'basic',
+ paintStyle: { strokeWidth: 2, stroke: '#2d8cf0' },
+ HoverPaintStyle: {stroke: '#ccc', strokeWidth: 3}
+ })
+ })
+
+ // Update remove connections
+ let currentConnects = plumbIns.getAllConnections()
+ let len = currentConnects.length
+ for (let i = 0; i < len; i++) {
+ if (this.preTasksToDelete.indexOf(currentConnects[i].sourceId) > -1 && currentConnects[i].targetId == targetId) {
+ plumbIns.deleteConnection(currentConnects[i])
+ i -= 1
+ len -= 1
+ }
+ }
+ }
+ }
$(`#${this.id}`).find('span').text(this.name)
this.conditionResult.successNode[0] = this.successBranch
@@ -562,6 +644,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
+ waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
status: this.status,
@@ -684,6 +767,16 @@
}
this.cacheBackfillItem = JSON.parse(JSON.stringify(o))
this.isContentBox = true
+
+ // Init value of preTask selector
+ let preTaskIds = $(`#${this.id}`).attr('data-targetarr')
+ if (!_.isEmpty(this.backfillItem)) {
+ if (preTaskIds && preTaskIds.length) {
+ this.backfillItem.preTasks = preTaskIds.split(',')
+ } else {
+ this.backfillItem.preTasks = []
+ }
+ }
},
mounted () {
let self = this
@@ -719,6 +812,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
+ waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
successBranch: this.successBranch,
@@ -744,8 +838,10 @@
mConditions,
mSelectInput,
mTimeoutAlarm,
+ mDependentTimeout,
mPriority,
- mWorkerGroups
+ mWorkerGroups,
+ mPreTasks,
}
}
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/pre_tasks.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/pre_tasks.vue
new file mode 100644
index 0000000000..adf889e958
--- /dev/null
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/pre_tasks.vue
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+
+
+
+
+ {{$t('Pre tasks')}}
+
+
+
+
+
+
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js
index 8a13aeacb4..e917cf3af0 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js
@@ -63,6 +63,9 @@ const stateType = [
}, {
code: 'DELAY_EXECUTION',
label: `${i18n.$t('Delay execution')}`
+ }, {
+ code: 'FORCED_SUCCESS',
+ label: `${i18n.$t('Forced success')}`
}
]
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue
index 01b478f6bd..3caf30ac62 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue
@@ -124,7 +124,6 @@
@@ -134,14 +133,12 @@
{{$t('Confirm')}}
- {{$t('Delete')}}
+ {{$t('Delete')}}
-
- {{$t('Export')}}
- {{$t('Batch copy')}}
- {{$t('Batch move')}}
-
+ {{$t('Export')}}
+ {{$t('Batch copy')}}
+ {{$t('Batch move')}}
@@ -268,6 +265,12 @@
* Close the delete layer
*/
_closeDelete (i) {
+ // close batch
+ if (i < 0) {
+ this.$refs['poptipDeleteAll'].doClose()
+ return
+ }
+ // close one
this.$refs[`poptip-delete-${i}`][0].doClose()
},
/**
@@ -596,8 +599,10 @@
}).then(res => {
this._onUpdate()
this.checkAll = false
+ this.strSelectIds = ''
this.$message.success(res.msg)
}).catch(e => {
+ this.strSelectIds = ''
this.checkAll = false
this.$message.error(e.msg || '')
})
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js
index 308af45a48..877f349803 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js
@@ -37,7 +37,7 @@ const warningTypeList = [
]
const isEmial = (val) => {
- let regEmail = /^([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+@([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+\.[a-zA-Z]{2,3}$/ // eslint-disable-line
+ let regEmail = /^([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+@([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+\.[a-zA-Z]{2,}$/ // eslint-disable-line
return regEmail.test(val)
}
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue
index dc1acf5c7f..cda59968cb 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue
@@ -288,7 +288,6 @@
@@ -298,7 +297,7 @@
{{$t('Confirm')}}
- {{$t('Delete')}}
+ {{$t('Delete')}}
@@ -344,6 +343,12 @@
* Close the delete layer
*/
_closeDelete (i) {
+ // close batch
+ if (i < 0) {
+ this.$refs['poptipDeleteAll'].doClose()
+ return
+ }
+ // close one
this.$refs[`poptip-delete-${i}`][0].doClose()
},
/**
@@ -539,9 +544,11 @@
}).then(res => {
this._onUpdate()
this.checkAll = false
+ this.strDelete = ''
this.$message.success(res.msg)
}).catch(e => {
this.checkAll = false
+ this.strDelete = ''
this.$message.error(e.msg || '')
})
}
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue
index cdcf0b0785..a6c7de1a1f 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue
@@ -91,6 +91,16 @@
{{item.duration}} |
{{item.retryTimes}} |
+
+
`
@@ -156,6 +168,17 @@
}
})
},
+ _forceSuccess (item) {
+ this.forceTaskSuccess({taskInstanceId: item.id}).then(res => {
+ if (res.code === 0) {
+ this.$message.success(res.msg)
+ } else {
+ this.$message.error(res.msg)
+ }
+ }).catch(e => {
+ this.$message.error(e.msg)
+ })
+ },
_go (item) {
this.$router.push({ path: `/projects/instance/list/${item.processInstanceId}` })
},
diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue b/dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue
index af296c9f28..132f6d9d24 100644
--- a/dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue
+++ b/dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue
@@ -85,7 +85,8 @@
userPassword: this.userPassword,
tenantId: this.userInfo.tenantId,
email: this.userInfo.email,
- phone: this.userInfo.phone
+ phone: this.userInfo.phone,
+ state: this.userInfo.state
}
this.spinnerLoading = true
this.updateUser(param).then(res => {
diff --git a/dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js b/dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js
index 93fab5a224..b35d07052a 100644
--- a/dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js
+++ b/dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js
@@ -734,6 +734,18 @@ export default {
})
})
},
+ /**
+ * Force fail/kill/need_fault_tolerance task success
+ */
+ forceTaskSuccess ({ state }, payload) {
+ return new Promise((resolve, reject) => {
+ io.post(`projects/${state.projectName}/task-instance/force-success`, payload, res => {
+ resolve(res)
+ }).catch(e => {
+ reject(e)
+ })
+ })
+ },
/**
* Query task record list
*/
diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js
index e1ecedc9ae..0f59aecc5a 100755
--- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js
+++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js
@@ -32,6 +32,7 @@ export default {
'Current node settings': 'Current node settings',
'View history': 'View history',
'View log': 'View log',
+ 'Force success': 'Force success',
'Enter this child node': 'Enter this child node',
'Node name': 'Node name',
'Run flag': 'Run flag',
@@ -426,8 +427,13 @@ export default {
'Timeout alarm': 'Timeout alarm',
'Timeout failure': 'Timeout failure',
'Timeout period': 'Timeout period',
+ 'Waiting Dependent complete': 'Waiting Dependent complete',
+ 'Waiting Dependent start': 'Waiting Dependent start',
+ 'Check interval': 'Check interval',
+ 'Timeout must be longer than check interval': 'Timeout must be longer than check interval',
'Timeout strategy must be selected': 'Timeout strategy must be selected',
'Timeout must be a positive integer': 'Timeout must be a positive integer',
+ 'Forced success': 'Forced success',
'Add dependency': 'Add dependency',
and: 'and',
or: 'or',
@@ -642,5 +648,6 @@ export default {
'Related items': 'Related items',
'Project name is required': 'Project name is required',
'Batch move': 'Batch move',
- Version: 'Version'
+ Version: 'Version',
+ 'Pre tasks': 'Pre tasks',
}
diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js
index f69142d3b5..0481cd0137 100755
--- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js
+++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js
@@ -32,6 +32,7 @@ export default {
'Current node settings': '当前节点设置',
'View history': '查看历史',
'View log': '查看日志',
+ 'Force success': '强制成功',
'Enter this child node': '进入该子节点',
'Node name': '节点名称',
'Please enter name (required)': '请输入名称(必填)',
@@ -420,8 +421,12 @@ export default {
'Timeout alarm': '超时告警',
'Timeout failure': '超时失败',
'Timeout period': '超时时长',
+ 'Waiting Dependent complete': '等待依赖完成',
+ 'Waiting Dependent start': '等待依赖启动',
+ 'Check interval': '检查间隔',
'Timeout strategy must be selected': '超时策略必须选一个',
'Timeout must be a positive integer': '超时时长必须为正整数',
+ 'Timeout must be longer than check interval': '超时时间必须比检查间隔长',
'Add dependency': '添加依赖',
and: '且',
or: '或',
@@ -432,6 +437,7 @@ export default {
Running: '正在运行',
'Waiting for dependency to complete': '等待依赖完成',
'Delay execution': '延时执行',
+ 'Forced success': '强制成功过',
Selected: '已选',
CurrentHour: '当前小时',
Last1Hour: '前1小时',
@@ -642,5 +648,6 @@ export default {
'Related items': '关联项目',
'Project name is required': '项目名称必填',
'Batch move': '批量移动',
- Version: '版本'
+ Version: '版本',
+ 'Pre tasks': '前置任务',
}
diff --git a/e2e/pom.xml b/e2e/pom.xml
index a5bdf69b41..fffbf8974e 100644
--- a/e2e/pom.xml
+++ b/e2e/pom.xml
@@ -87,6 +87,16 @@
commons-pool2
${commons-pool2.version}
+
+ org.jyaml
+ jyaml
+ 1.3
+
+
+ org.springframework
+ spring-core
+ 5.1.5.RELEASE
+
diff --git a/e2e/src/main/java/org/apache/dolphinscheduler/util/YmlReader.java b/e2e/src/main/java/org/apache/dolphinscheduler/util/YmlReader.java
new file mode 100644
index 0000000000..e2784eb0ff
--- /dev/null
+++ b/e2e/src/main/java/org/apache/dolphinscheduler/util/YmlReader.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+
+import org.ho.yaml.Yaml;
+import org.springframework.core.io.DefaultResourceLoader;
+import org.springframework.core.io.Resource;
+
+/**
+ * read yml file
+ */
+public class YmlReader {
+ public static HashMap> map;
+ public String getDataYml(String filePath, String key1, String key2) {
+ Yaml yaml = new Yaml();
+ Resource resource = new DefaultResourceLoader().getResource("classpath:" + filePath + ".yml");
+ try {
+ InputStream inputStream = resource.getInputStream();
+ map = yaml.loadType(inputStream, HashMap.class);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ String data = map.get(key1).get(key2);
+ return data;
+ }
+}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java
index 23c98b4e46..73a8da2d62 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java
@@ -23,7 +23,6 @@ import org.openqa.selenium.PageLoadStrategy;
import org.openqa.selenium.WebDriver;
import org.openqa.selenium.chrome.ChromeDriver;
import org.openqa.selenium.chrome.ChromeOptions;
-import org.openqa.selenium.remote.CapabilityType;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java
index e56df5a448..332b8000b6 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java
@@ -28,7 +28,6 @@ public class LoginData {
*/
public static final String URL = PropertiesReader.getKey("LOGIN_URL");
-
/**
* Login username
*/
@@ -38,6 +37,4 @@ public class LoginData {
* Login password
*/
public static final String PASSWORD = PropertiesReader.getKey("PASSWORD");
-
- public static final String TENANT = "租户管理 - DolphinScheduler";
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java
index 7114851d25..fa827f6707 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java
@@ -16,10 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
-public class ProcessInstanceData {
- //Process Instance page title
- public static final String PROCESS_INSTANCE_TITLE = "工作流实例 - DolphinScheduler";
- public static final String RERUN_TYPE= "重跑";
-
+import org.apache.dolphinscheduler.util.YmlReader;
+public class ProcessInstanceData {
+ public String getProcessInstanceData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String processInstanceData = ymlReader.getDataYml("testData/workflow_zh_cn", "processInstance", param);
+ return processInstanceData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java
index 036a6771bd..22b44684bb 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java
@@ -16,14 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
-public class ProjectData {
-
- public static final String CREATE_PROJECT_BUTTON = "创建项目";
+import org.apache.dolphinscheduler.util.YmlReader;
- // create project name
- public static final String PROJECT_NAME = "selenium_project_1";
- // create project description
- public static final String DESCRIPTION = "test create project description";
- // project page title
- public static final String PROJECT_TITLE = "项目 - DolphinScheduler";
+public class ProjectData {
+ public String getProjectData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String projectData = ymlReader.getDataYml("testData/workflow_zh_cn", "projectManage", param);
+ return projectData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java
index 70112edd7a..7653f5a09d 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java
@@ -16,17 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
-public class RunWorkflowData {
- /**
- * run workflow data
- */
- //input shell task name
- public static final String RECIPIENT = "123456789@qq.com";
-
- //input shell task description
- public static final String Cc = "qwe12312sds@qq.com";
-
- public static final String RUN_WORKFLOW_TITLE = "工作流定义 - DolphinScheduler";
-
+import org.apache.dolphinscheduler.util.YmlReader;
+public class RunWorkflowData {
+ public String getRunWorkflowData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String runWorkflowData = ymlReader.getDataYml("testData/workflow_zh_cn", "runWorkflow", param);
+ return runWorkflowData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java
index e76852dd0c..87fbbd8c03 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java
@@ -16,23 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
-public class TimingData {
- /**
- * timing data
- */
- //input shell task name
- public static final String RECIPIENT = "test123456@qq.com";
-
- //input shell task description
- public static final String Cc = "test.123qwe@qq.com";
-
- public static final String EDIT_RECIPIENT = "test.edit123456@qq.com";
-
- public static final String EDIT_Cc = "test.edit123qwe@qq.com";
+import org.apache.dolphinscheduler.util.YmlReader;
- public static final String TIMING_OFFLINE_STATE = "下线";
-
- public static final String TIMING_ONLINE_STATE = "上线";
-
- public static final String TIMING_TITLE = "定时任务列表 - DolphinScheduler";
+public class TimingData {
+ public String getTimingData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String timingData = ymlReader.getDataYml("testData/workflow_zh_cn", "timing", param);
+ return timingData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java
index c9eb32e948..41b33e76fc 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java
@@ -16,74 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
-public class WorkflowDefineData {
- /**
- * create workflow data
- */
-
- public static final String workflow_define = "工作流定义";
-
- //input shell task name
- public static final String SHELL_TASK_NAME = "shell_task_selenium_1";
-
- //input shell task description
- public static final String SHELL_TASK_DESCRIPTION = "shell task description test";
-
- //input timeout
- public static final String INPUT_TIMEOUT = "60";
-
- //input shell script
- public static final String SHELL_SCRIPT = "echo 1111111";
-
- //input custom parameters
- public static final String INPUT_CUSTOM_PARAMETERS = "selenium_parameter";
-
- //input custom parameters value
- public static final String INPUT_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_123";
-
- //input add custom parameters
- public static final String INPUT_ADD_CUSTOM_PARAMETERS = "selenium_parameter_delete";
-
- //input add custom parameters value
- public static final String INPUT_ADD_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_delete_456";
-
- //workflow define title
- public static final String WORKFLOW_TITLE = "工作流定义 - DolphinScheduler";
-
- //create workflow title
- public static final String CREATE_WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler";
-
-
- /**
- * save workflow data
- */
- //input workflow name
- public static final String INPUT_WORKFLOW_NAME = "selenium_shell_1";
-
- //input workflow description
- public static final String INPUT_WORKFLOW_DESCRIPTION = "test selenium_shell_1 description";
-
- //input workflow timeout
- public static final String INPUT_WORKFLOW_TIMEOUT = "30";
-
- //input workflow global parameters
- public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_1";
-
- //input workflow global parameters value
- public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_1";
-
- //input to add workflow global parameters
- public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_2";
-
- //input to add workflow global parameters value
- public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_2";
-
- /**
- * workflowDefine list
- */
- public static final String WORKFLOW_ONLINE_STATE = "上线";
-
- public static final String WORKFLOW_OFFLINE_STATE = "下线";
-
+import org.apache.dolphinscheduler.util.YmlReader;
+public class WorkflowDefineData {
+ public String getWorkflowDefineData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String workflowDefineData = ymlReader.getDataYml("testData/workflow_zh_cn", "workflowDefine", param);
+ return workflowDefineData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java
index ae996a209d..6bb2db4528 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java
@@ -16,17 +16,13 @@
*/
package org.apache.dolphinscheduler.data.security;
-public class AlertManageData {
- //Alert Name
- public static final String ALERT_NAME = "selenium_alert_Name";
- public static final String CREATE_ALERT = "创建告警组";
-
- // Alert Type
- public static final String ALERT_TYPE = "邮件";
+import org.apache.dolphinscheduler.util.YmlReader;
- //Alert Description
- public static final String DESCRIPTION = "create alert test";
-
- public static final String ALERT_MANAGE = "告警组管理 - DolphinScheduler";
+public class AlertManageData {
+ public String getAlertData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String alertData = ymlReader.getDataYml("testData/security_zh_cn", "alertManage", param);
+ return alertData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java
index 46f0d834d1..4b721602be 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java
@@ -16,29 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
-public class QueueManageData {
- /**
- * Create Queue Name
- */
- public static final String QUEUE_NAME = "selenium_queue_name";
-
- public static final String CREATE_QUEUE = "创建队列";
-
- /**
- * Create Queue Value
- */
- public static final String QUEUE_VALUE = "selenium_queue_value";
-
- /**
- * Edit Queue Name
- */
- public static final String EDIT_QUEUE_NAME = "selenium_queue_value_edit";
-
- /**
- * Edit Queue Value
- */
- public static final String EDIT_QUEUE_VALUE = "selenium_queue_value_edit";
-
- public static final String QUEUE_MANAGE = "队列管理 - DolphinScheduler";
+import org.apache.dolphinscheduler.util.YmlReader;
+public class QueueManageData {
+ public String getQueueData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String queueData = ymlReader.getDataYml("testData/security_zh_cn", "queueManage", param);
+ return queueData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java
index 71e50e7150..0c9727799d 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java
@@ -20,30 +20,12 @@
*/
package org.apache.dolphinscheduler.data.security;
+import org.apache.dolphinscheduler.util.YmlReader;
public class TenantManageData {
- /**
- * Tenant Code
- */
- public static final String TENANT_CODE = "selenium_tenant_code_1";
-
- /**
- * Tenant Name
- */
- public static final String TENANT_NAME = "selenium_tenant_Name";
-
- /**
- * Queue
- */
- public static final String QUEUE = "default";
-
- /**
- * Description
- */
- public static final String DESCRIPTION = "create tenant test";
-
- public static final String TENANT_MANAGE = "租户管理 - DolphinScheduler";
-
-
-
+ public String getTenantData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String tenantData = ymlReader.getDataYml("testData/security_zh_cn", "tenantManage", param);
+ return tenantData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java
index 9dd625ec25..10fdc25795 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java
@@ -16,13 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
-public class TokenManageData {
- public static final String TOKEN_MANAGE = "令牌管理 - DolphinScheduler";
- public static final String CREATE_TOKEN = "创建令牌";
-
- public static final String DATE = "2038-06-10 00:00:00";
- public static final String NAME = "admin";
-
-
+import org.apache.dolphinscheduler.util.YmlReader;
+public class TokenManageData {
+ public String getTokenData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String tokenData = ymlReader.getDataYml("testData/security_zh_cn", "tokenManage", param);
+ return tokenData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java
index ee7fc60bc4..699cce4568 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java
@@ -16,29 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
+import org.apache.dolphinscheduler.util.YmlReader;
public class UserManageData {
- /**
- * create user
- */
- public static final String USERNAME = "selenium_user_1";
-
- public static final String PASSWORD = "123456qwe";
-
- public static final String EMAIL = "123456789@qq.com";
-
- public static final String PHONE = "15811112222";
-
- public static final String USER_MANAGE = "用户管理 - DolphinScheduler";
-
- public static final String CREATE_USER_BUTTON = "创建用户";
-
-
- /**
- * edit user
- */
- public static final String EDIT_USERNAME = "selenium_user_edit";
- public static final String EDIT_PASSWORD = "123456qwe";
- public static final String EDIT_EMAIL = "123456_edit@qq.com";
- public static final String EDIT_PHONE = "15800001111";
+ public String getUserData(String param) {
+ YmlReader ymlReader = new YmlReader();
+ String userData = ymlReader.getDataYml("testData/security_zh_cn", "userManage", param);
+ return userData;
+ }
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java
index 01fdb39e4a..c851be3d0d 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java
@@ -179,6 +179,8 @@ public class WorkflowDefineLocator {
//scroll to element bottom
public static final By SCROLL_BOTTOM = By.xpath("//span/a/em");
+ public static final By WORKFLOW_NAME = By.xpath("//table/tr[2]/td[3]/span/a");
+
/**
* online workflow
*/
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java
index d935dd54a0..e85807fa68 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java
@@ -24,18 +24,18 @@ public class TokenManageLocator {
public static final By CLICK_CREATE_TOKEN = By.xpath("//div[2]/div/div[2]/div[2]/div/div[1]/button/span");
+ public static final By CREATE_TOKEN_POPUP = By.xpath("//div[5]/div/div[2]/div/div[1]/span");
+
public static final By SELECT_USER = By.xpath("//div[2]/div[2]/div/div/div/span/i");
public static final By CLICK_GENERATE_TOKEN_BUTTON = By.xpath("//div[3]/div[2]/button/span");
public static final By CLICK_SUBMIT_BUTTON = By.xpath("//div[3]/button[2]/span");
- public static final By EDIT_TOKEN_BUTTON = By.xpath("//table/tr[2]/td[7]/button");
-
//edit token
public static final By TOKEN = By.xpath("//table/tr[2]/td[1]/span");
+ public static final By EDIT_TOKEN_BUTTON = By.xpath("//table/tr[2]/td[7]/button");
- public static final By CLICK_EDIT_BUTTON = By.xpath("//div[3]/div[1]/div/table/tr[2]/td[7]/button/i");
//delete token
public static final By CLICK_DELETE_BUTTON = By.xpath("//div[3]/div[1]/div/table/tr[2]/td[7]/span/button");
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java
index ecfd285d60..80f9df20c0 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java
@@ -49,6 +49,8 @@ public class UserManageLocator {
public static final By SUBMIT = By.xpath("//div[3]/button[2]/span");
+ public static final By USERNAME = By.xpath("//table/tr[2]/td[2]/span");
+
/**
* edit user
*/
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java
index 39bd152d0a..468938d342 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java
@@ -18,13 +18,14 @@ package org.apache.dolphinscheduler.page;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.data.LoginData;
+import org.apache.dolphinscheduler.data.security.TenantManageData;
import org.apache.dolphinscheduler.locator.LoginLocator;
import org.openqa.selenium.Cookie;
import org.openqa.selenium.WebDriver;
-
-
public class LoginPage extends PageCommon {
+ TenantManageData tenantManageData = new TenantManageData();
+
/**
* Unique constructor
* @param driver driver
@@ -33,7 +34,6 @@ public class LoginPage extends PageCommon {
super(driver);
}
-
/**
* jump page
*/
@@ -65,6 +65,6 @@ public class LoginPage extends PageCommon {
moveToElement(LoginLocator.LOGIN_BUTTON_MOVE);
// Whether to enter the specified page after login
- return ifTitleContains(LoginData.TENANT);
+ return ifTitleContains(tenantManageData.getTenantData("tenantTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java
index 8c251afc84..04124b467f 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java
@@ -23,6 +23,8 @@ import org.apache.dolphinscheduler.data.project.ProcessInstanceData;
public class ProcessInstancePage extends PageCommon {
+ ProcessInstanceData processInstanceData = new ProcessInstanceData();
+
public ProcessInstancePage(WebDriver driver) {
super(driver);
}
@@ -36,7 +38,7 @@ public class ProcessInstancePage extends PageCommon {
clickTopElement(ProcessInstanceLocator.CLICK_PROCESS_INSTANCE_NAME);
locateElement(ProcessInstanceLocator.PROCESS_INSTANCE_SUCCESS_STATE);
clickTopElement(ProcessInstanceLocator.CLICK_RERUN_BUTTON);
- return ifTitleContains(ProcessInstanceData.PROCESS_INSTANCE_TITLE);
+ return ifTitleContains(processInstanceData.getProcessInstanceData("processInstanceTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java
index 6a93094340..a450ec7121 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java
@@ -17,12 +17,13 @@
package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
-import org.apache.dolphinscheduler.constant.TestConstant;
import org.apache.dolphinscheduler.data.project.ProjectData;
import org.apache.dolphinscheduler.locator.project.ProjectLocator;
import org.openqa.selenium.WebDriver;
public class ProjectPage extends PageCommon {
+ ProjectData projectData = new ProjectData();
+
public ProjectPage(WebDriver driver) {
super(driver);
}
@@ -32,7 +33,7 @@ public class ProjectPage extends PageCommon {
*/
public boolean jumpProjectManagePage() throws InterruptedException {
clickTopElement(ProjectLocator.PROJECT_MANAGE);
- return ifTitleContains(ProjectData.PROJECT_TITLE);
+ return ifTitleContains(projectData.getProjectData("projectTitle"));
}
/**
@@ -41,18 +42,18 @@ public class ProjectPage extends PageCommon {
* @return Whether to enter the specified page after create project
*/
public boolean createProject() throws InterruptedException {
- ifTextExists(ProjectLocator.CREATE_PROJECT_BUTTON,ProjectData.CREATE_PROJECT_BUTTON);
+ ifTextExists(ProjectLocator.CREATE_PROJECT_BUTTON, projectData.getProjectData("createProjectButton"));
clickElement(ProjectLocator.CREATE_PROJECT_BUTTON);
// input create project data
- sendInput(ProjectLocator.PROJECT_NAME, ProjectData.PROJECT_NAME);
- sendInput(ProjectLocator.PROJECT_DESCRIPTION, ProjectData.DESCRIPTION);
+ sendInput(ProjectLocator.PROJECT_NAME, projectData.getProjectData("projectName"));
+ sendInput(ProjectLocator.PROJECT_DESCRIPTION, projectData.getProjectData("description"));
// click submit button
clickButton(ProjectLocator.SUBMIT_BUTTON);
// Whether to enter the specified page after submit
- return ifTextExists(ProjectLocator.LIST_PROJECT_NAME,ProjectData.PROJECT_NAME);
+ return ifTextExists(ProjectLocator.LIST_PROJECT_NAME, projectData.getProjectData("projectName"));
}
/**
@@ -68,6 +69,6 @@ public class ProjectPage extends PageCommon {
clickElement(ProjectLocator.CONFIRM_DELETE_PROJECT_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(ProjectData.PROJECT_TITLE);
+ return ifTitleContains(projectData.getProjectData("projectTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java
index 921e593e44..c4b8a68dee 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java
@@ -24,13 +24,16 @@ import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class RunWorkflowPage extends PageCommon {
+ RunWorkflowData runWorkflowData = new RunWorkflowData();
+ WorkflowDefineData workflowDefineData = new WorkflowDefineData();
+
public RunWorkflowPage(WebDriver driver) {
super(driver);
}
public boolean runWorkflow() throws InterruptedException {
// Determine whether the workflow status is online
- ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, WorkflowDefineData.WORKFLOW_ONLINE_STATE);
+ ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, runWorkflowData.getRunWorkflowData("online"));
// click run workflow button
System.out.println("Click run workflow button");
@@ -46,10 +49,10 @@ public class RunWorkflowPage extends PageCommon {
clickElement(RunWorkflowLocator.SELECT_WORKER_GROUP);
clickElement(RunWorkflowLocator.CLICK_NOTICE_GROUP);
clickElement(RunWorkflowLocator.SELECT_NOTICE_GROUP);
- sendInput(RunWorkflowLocator.INPUT_RECIPIENT, RunWorkflowData.RECIPIENT);
- sendInput(RunWorkflowLocator.INPUT_Cc,RunWorkflowData.Cc);
+ sendInput(RunWorkflowLocator.INPUT_RECIPIENT, runWorkflowData.getRunWorkflowData("recipient"));
+ sendInput(RunWorkflowLocator.INPUT_Cc, runWorkflowData.getRunWorkflowData("Cc"));
clickButton(RunWorkflowLocator.CLICK_RUNNING_BUTTON);
- return ifTitleContains(RunWorkflowData.RUN_WORKFLOW_TITLE);
+ return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java
index 3febc545ff..37f3fdab2a 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java
@@ -18,23 +18,24 @@ package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.data.project.TimingData;
-import org.apache.dolphinscheduler.data.project.WorkflowDefineData;
import org.apache.dolphinscheduler.locator.project.TimingLocator;
import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class TimingPage extends PageCommon {
+ TimingData timingData = new TimingData();
+
public TimingPage(WebDriver driver) {
super(driver);
}
-
/**
* create timing
*/
public boolean createTiming() throws InterruptedException {
+ flushPage();
// Determine whether the workflow status is online
- ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, WorkflowDefineData.WORKFLOW_ONLINE_STATE);
+ ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, timingData.getTimingData("online"));
// click timing button
System.out.println("Click timing button");
@@ -52,11 +53,11 @@ public class TimingPage extends PageCommon {
clickElement(TimingLocator.SELECT_WORKER_GROUP);
clickElement(TimingLocator.CLICK_NOTICE_GROUP);
clickElement(TimingLocator.SELECT_NOTICE_GROUP);
- sendInput(TimingLocator.INPUT_RECIPIENT, TimingData.RECIPIENT);
- sendInput(TimingLocator.INPUT_Cc,TimingData.Cc);
+ sendInput(TimingLocator.INPUT_RECIPIENT, timingData.getTimingData("recipient"));
+ sendInput(TimingLocator.INPUT_Cc, timingData.getTimingData("Cc"));
clickButton(TimingLocator.CLICK_CREATE_BUTTON);
- return ifTextExists(TimingLocator.TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
+ return ifTextExists(TimingLocator.TIMING_STATE, timingData.getTimingData("offline"));
}
/**
@@ -68,7 +69,7 @@ public class TimingPage extends PageCommon {
clickButton(TimingLocator.CLICK_TIMING_MANAGEMENT_BUTTON);
// Determine whether the workflow name exists
- ifTextExists(TimingLocator.WORKFLOW_NAME, WorkflowDefineData.INPUT_WORKFLOW_NAME);
+ ifTextExists(TimingLocator.WORKFLOW_NAME, timingData.getTimingData("offline"));
System.out.println("Click edit timing button");
clickButton(TimingLocator.CLICK_EDIT_TIMING_BUTTON);
@@ -85,11 +86,11 @@ public class TimingPage extends PageCommon {
clickElement(TimingLocator.SELECT_WORKER_GROUP);
clickElement(TimingLocator.CLICK_NOTICE_GROUP);
clickElement(TimingLocator.SELECT_NOTICE_GROUP);
- sendInput(TimingLocator.INPUT_RECIPIENT, TimingData.EDIT_RECIPIENT);
- sendInput(TimingLocator.INPUT_Cc,TimingData.EDIT_Cc);
+ sendInput(TimingLocator.INPUT_RECIPIENT, timingData.getTimingData("editRecipient"));
+ sendInput(TimingLocator.INPUT_Cc, timingData.getTimingData("editCc"));
clickButton(TimingLocator.CLICK_CREATE_BUTTON);
- return ifTitleContains(TimingData.TIMING_TITLE );
+ return ifTitleContains(timingData.getTimingData("timingTitle"));
}
@@ -99,13 +100,13 @@ public class TimingPage extends PageCommon {
public boolean onlineTiming() throws InterruptedException {
flushPage();
// Determine whether the timing is offline
- ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
+ ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
// click online timing button
System.out.println("Click online timing button");
clickElement(TimingLocator.CLICK_ONLINE_TIMING_BUTTON);
- return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_ONLINE_STATE);
+ return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("online"));
}
@@ -115,13 +116,13 @@ public class TimingPage extends PageCommon {
public boolean offlineTiming() throws InterruptedException {
flushPage();
// Determine whether the timing is online
- ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_ONLINE_STATE);
+ ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("online"));
// click offline timing button
System.out.println("Click offline timing button");
clickElement(TimingLocator.CLICK_OFFLINE_TIMING_BUTTON);
- return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
+ return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
}
@@ -131,7 +132,7 @@ public class TimingPage extends PageCommon {
*/
public boolean deleteTiming() throws InterruptedException {
// Determine whether the timing is offline
- ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
+ ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
// click offline timing button
System.out.println("Click delete timing button");
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java
index 8826222189..72722d8f10 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java
@@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
-import org.apache.dolphinscheduler.constant.TestConstant;
import org.apache.dolphinscheduler.data.project.ProjectData;
import org.apache.dolphinscheduler.data.project.WorkflowDefineData;
import org.apache.dolphinscheduler.locator.project.ProjectLocator;
@@ -25,6 +24,9 @@ import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class WorkflowDefinePage extends PageCommon {
+ WorkflowDefineData workflowDefineData = new WorkflowDefineData();
+ ProjectData projectData = new ProjectData();
+
public WorkflowDefinePage(WebDriver driver) {
super(driver);
}
@@ -34,18 +36,18 @@ public class WorkflowDefinePage extends PageCommon {
*/
public boolean jumpWorkflowPage() throws InterruptedException {
- ifTextExists(ProjectLocator.LIST_PROJECT_NAME, ProjectData.PROJECT_NAME);
+ ifTextExists(ProjectLocator.LIST_PROJECT_NAME, projectData.getProjectData("projectName"));
// click project name
clickElement(WorkflowDefineLocator.CLICK_PROJECT_NAME);
- ifTextExists(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE,WorkflowDefineData.workflow_define);
+ ifTextExists(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE,workflowDefineData.getWorkflowDefineData("workflowDefine"));
System.out.println("Click on workflow define to jump to workflow define page");
// click workflow define
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
- return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
+ return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
public boolean createWorkflow() throws InterruptedException {
@@ -58,8 +60,7 @@ public class WorkflowDefinePage extends PageCommon {
dragAndDrop(WorkflowDefineLocator.MOUSE_DOWN_AT_SHELL, WorkflowDefineLocator.MOUSE_MOVE_SHELL_AT_DAG);
//input shell task _name
- sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_NAME , WorkflowDefineData.SHELL_TASK_NAME);
-
+ sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_NAME, workflowDefineData.getWorkflowDefineData("shellTaskName"));
//click stop run type
clickElement(WorkflowDefineLocator.CLICK_STOP_RUN_TYPE);
@@ -67,7 +68,7 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_NORMAL_RUN_TYPE);
//input shell task description
- sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_DESCRIPTION , WorkflowDefineData.SHELL_TASK_DESCRIPTION);
+ sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_DESCRIPTION, workflowDefineData.getWorkflowDefineData("shellTaskDescription"));
//select task priority
clickElement(WorkflowDefineLocator.CLICK_TASK_PRIORITY);
@@ -100,20 +101,20 @@ public class WorkflowDefinePage extends PageCommon {
clearInput(WorkflowDefineLocator.SELECT_TIMEOUT);
//input timeout
- sendInput(WorkflowDefineLocator.SELECT_TIMEOUT, WorkflowDefineData.INPUT_TIMEOUT);
+ sendInput(WorkflowDefineLocator.SELECT_TIMEOUT, workflowDefineData.getWorkflowDefineData("taskTimeout"));
//click codeMirror and input script
- inputCodeMirror(WorkflowDefineLocator.CLICK_CODE_MIRROR, WorkflowDefineLocator.INPUT_SCRIPT, WorkflowDefineData.SHELL_SCRIPT);
+ inputCodeMirror(WorkflowDefineLocator.CLICK_CODE_MIRROR, WorkflowDefineLocator.INPUT_SCRIPT, workflowDefineData.getWorkflowDefineData("shellScript"));
scrollToElementBottom(WorkflowDefineLocator.SCROLL_BOTTOM);
//click custom parameters
clickElement(WorkflowDefineLocator.CLICK_CUSTOM_PARAMETERS);
//input custom parameters
- sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS, WorkflowDefineData.INPUT_CUSTOM_PARAMETERS);
+ sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS, workflowDefineData.getWorkflowDefineData("customParameter1"));
//input custom parameters value
- sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS_VALUE, WorkflowDefineData.INPUT_CUSTOM_PARAMETERS_VALUE);
+ sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS_VALUE, workflowDefineData.getWorkflowDefineData("customParameterValue1"));
//click add custom parameters
clickElement(WorkflowDefineLocator.CLICK_ADD_CUSTOM_PARAMETERS);
@@ -121,10 +122,10 @@ public class WorkflowDefinePage extends PageCommon {
scrollToElementBottom(WorkflowDefineLocator.SCROLL_BOTTOM);
//input add custom parameters
- sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS, WorkflowDefineData.INPUT_ADD_CUSTOM_PARAMETERS);
+ sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS, workflowDefineData.getWorkflowDefineData("customParameter2"));
//input add custom parameters value
- sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE, WorkflowDefineData.INPUT_ADD_CUSTOM_PARAMETERS_VALUE);
+ sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE, workflowDefineData.getWorkflowDefineData("customParameterValue2"));
//click delete custom parameters
clickElement(WorkflowDefineLocator.CLICK_DELETE_CUSTOM_PARAMETERS);
@@ -140,7 +141,7 @@ public class WorkflowDefinePage extends PageCommon {
clickButton(WorkflowDefineLocator.COPY_TASK);
clickButton(WorkflowDefineLocator.CLICK_LINE);
mouseMovePosition(WorkflowDefineLocator.LINE_SOURCES_TASK,WorkflowDefineLocator.LINE_TARGET_TASK);
- return ifTitleContains(WorkflowDefineData.CREATE_WORKFLOW_TITLE);
+ return ifTitleContains(workflowDefineData.getWorkflowDefineData("createWorkflowTitle"));
}
/**
@@ -153,10 +154,10 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_SAVE_WORKFLOW_BUTTON);
//input workflow name
- sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_NAME, WorkflowDefineData.INPUT_WORKFLOW_NAME);
+ sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_NAME, workflowDefineData.getWorkflowDefineData("workflowName"));
//input workflow description
- sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_DESCRIPTION, WorkflowDefineData.INPUT_WORKFLOW_DESCRIPTION);
+ sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_DESCRIPTION, workflowDefineData.getWorkflowDefineData("workflowDescription"));
//select tenant
clickElement(WorkflowDefineLocator.CLICK_TENANT);
@@ -167,25 +168,25 @@ public class WorkflowDefinePage extends PageCommon {
clearInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT);
//input workflow timeout
- sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT, WorkflowDefineData.INPUT_WORKFLOW_TIMEOUT);
+ sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT, workflowDefineData.getWorkflowDefineData("workflowTimeout"));
//click workflow global parameters
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_GLOBAL_PARAMETERS);
//input workflow global parameters
- sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS, WorkflowDefineData.INPUT_WORKFLOW_GLOBAL_PARAMETERS);
+ sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS, workflowDefineData.getWorkflowDefineData("globalParameter1"));
//input workflow global parameters value
- sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES, WorkflowDefineData.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES);
+ sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES, workflowDefineData.getWorkflowDefineData("globalParameterValue1"));
//click to add workflow global parameters
clickElement(WorkflowDefineLocator.CLICK_ADD_WORKFLOW_GLOBAL_PARAMETERS);
//input to add workflow global parameters
- sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS, WorkflowDefineData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS);
+ sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS, workflowDefineData.getWorkflowDefineData("globalParameter2"));
//input to add workflow global parameters value
- sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES, WorkflowDefineData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES);
+ sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES, workflowDefineData.getWorkflowDefineData("globalParameterValue2"));
//delete workflow global parameters value
clickElement(WorkflowDefineLocator.CLICK_DELETE_WORKFLOW_GLOBAL_PARAMETERS);
@@ -194,33 +195,33 @@ public class WorkflowDefinePage extends PageCommon {
System.out.println("submit workflow");
clickButton(WorkflowDefineLocator.CLICK_ADD_BUTTON);
- return ifTitleContains(WorkflowDefineData.CREATE_WORKFLOW_TITLE);
+ return ifTitleContains(workflowDefineData.getWorkflowDefineData("createWorkflowTitle"));
}
public boolean onlineWorkflow() throws InterruptedException {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is offline
- ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_OFFLINE_STATE);
+ ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
// click online button
System.out.println("Click online workflow button");
clickButton(WorkflowDefineLocator.CLICK_ONLINE_WORKFLOW_BUTTON);
- return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
+ return ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("online"));
}
public boolean offlineWorkflow() throws InterruptedException {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is online
- ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_ONLINE_STATE);
+ ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("online"));
// click offline button
System.out.println("offline workflow");
clickButton(WorkflowDefineLocator.CLICK_OFFLINE_WORKFLOW_BUTTON);
- return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
+ return ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
}
@@ -229,7 +230,7 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is offline
- ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_OFFLINE_STATE);
+ ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
clickButton(WorkflowDefineLocator.DELETE_WORKFLOW_BOTTOM);
@@ -237,6 +238,6 @@ public class WorkflowDefinePage extends PageCommon {
clickButton(WorkflowDefineLocator.CONFIRM_DELETE_WORKFLOW_BOTTOM);
// Whether to enter the specified page after submit
- return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
+ return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java
index 541251e0db..97e167ec29 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java
@@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.AlertManageLocator;
import org.openqa.selenium.WebDriver;
public class AlertManagePage extends PageCommon {
+ AlertManageData alertManageData = new AlertManageData();
+
/**
* Unique constructor
* @param driver driver
@@ -41,26 +43,26 @@ public class AlertManagePage extends PageCommon {
clickElement(AlertManageLocator.CLICK_ALERT_MANAGE);
//determine whether the create alert button exists
- ifTextExists(AlertManageLocator.CLICK_CREATE_ALERT,AlertManageData.CREATE_ALERT);
+ ifTextExists(AlertManageLocator.CLICK_CREATE_ALERT,alertManageData.getAlertData("createAlert"));
// click create alert button
System.out.println("start click create alert button");
clickElement(AlertManageLocator.CLICK_CREATE_ALERT);
// input alert data
System.out.println("start input alert ");
- sendInput(AlertManageLocator.INPUT_ALERT_NAME, AlertManageData.ALERT_NAME);
+ sendInput(AlertManageLocator.INPUT_ALERT_NAME, alertManageData.getAlertData("alertName"));
clickElement(AlertManageLocator.CLICK_ALERT_TYPE);
clickElement(AlertManageLocator.SELECT_ALERT_EMAIL);
- sendInput(AlertManageLocator.INPUT_ALERT_DESCRIPTION, AlertManageData.DESCRIPTION);
+ sendInput(AlertManageLocator.INPUT_ALERT_DESCRIPTION, alertManageData.getAlertData("description"));
// click button
clickButton(AlertManageLocator.SUBMIT_ALERT);
// Whether to enter the specified page after submit
- return ifTextExists(AlertManageLocator.ALERT_NAME, AlertManageData.ALERT_NAME);
+ return ifTextExists(AlertManageLocator.ALERT_NAME, alertManageData.getAlertData("alertName"));
}
public boolean deleteAlert() throws InterruptedException {
@@ -68,7 +70,7 @@ public class AlertManagePage extends PageCommon {
// click alert manage
clickElement(AlertManageLocator.CLICK_ALERT_MANAGE);
- ifTextExists(AlertManageLocator.ALERT_NAME, AlertManageData.ALERT_NAME);
+ ifTextExists(AlertManageLocator.ALERT_NAME, alertManageData.getAlertData("alertName"));
// click delete alert button
clickButton(AlertManageLocator.DELETE_ALERT_BUTTON);
@@ -77,6 +79,6 @@ public class AlertManagePage extends PageCommon {
clickButton(AlertManageLocator.CONFIRM_DELETE_ALERT_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(AlertManageData.ALERT_MANAGE);
+ return ifTitleContains(alertManageData.getAlertData("alertTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/QueueManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/QueueManagePage.java
index 368202e921..f6a55bb73b 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/QueueManagePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/QueueManagePage.java
@@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.QueueManageLocator;
import org.openqa.selenium.WebDriver;
public class QueueManagePage extends PageCommon {
+ QueueManageData queueManageData = new QueueManageData();
+
/**
* Unique constructor
* @param driver driver
@@ -41,7 +43,7 @@ public class QueueManagePage extends PageCommon {
clickElement(QueueManageLocator.CLICK_QUEUE_MANAGE);
//determine whether the create queue button exists
- ifTextExists(QueueManageLocator.CLICK_CREATE_QUEUE,QueueManageData.CREATE_QUEUE);
+ ifTextExists(QueueManageLocator.CLICK_CREATE_QUEUE, queueManageData.getQueueData("createQueueButton"));
// click create queue button
System.out.println("start click create queue button");
@@ -49,15 +51,15 @@ public class QueueManagePage extends PageCommon {
// input queue data
System.out.println("start input queue");
- sendInput(QueueManageLocator.INPUT_QUEUE_NAME, QueueManageData.QUEUE_NAME);
+ sendInput(QueueManageLocator.INPUT_QUEUE_NAME, queueManageData.getQueueData("queueName"));
- sendInput(QueueManageLocator.INPUT_QUEUE_VALUE, QueueManageData.QUEUE_VALUE);
+ sendInput(QueueManageLocator.INPUT_QUEUE_VALUE, queueManageData.getQueueData("queueValue"));
// click button
clickButton(QueueManageLocator.SUBMIT_QUEUE);
// Whether to enter the specified page after submit
- return ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, QueueManageData.QUEUE_NAME);
+ return ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, queueManageData.getQueueData("queueName"));
}
@@ -70,19 +72,19 @@ public class QueueManagePage extends PageCommon {
// click queue manage
clickElement(QueueManageLocator.CLICK_QUEUE_MANAGE);
- ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, QueueManageData.QUEUE_NAME);
+ ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, queueManageData.getQueueData("queueName"));
// click edit queue button
clickButton(QueueManageLocator.CLICK_EDIT_QUEUE);
// input queue data
- clearSendInput(QueueManageLocator.INPUT_QUEUE_NAME, QueueManageData.EDIT_QUEUE_NAME);
- clearSendInput(QueueManageLocator.INPUT_QUEUE_VALUE, QueueManageData.EDIT_QUEUE_VALUE);
+ clearSendInput(QueueManageLocator.INPUT_QUEUE_NAME, queueManageData.getQueueData("editQueueName"));
+ clearSendInput(QueueManageLocator.INPUT_QUEUE_VALUE, queueManageData.getQueueData("editQueueValue"));
// click button
clickButton(QueueManageLocator.SUBMIT_QUEUE);
// Whether to enter the specified page after submit
- return ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, QueueManageData.EDIT_QUEUE_NAME);
+ return ifTextExists(QueueManageLocator.LIST_QUEUE_NAME, queueManageData.getQueueData("editQueueName"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java
index 40d964fa33..44f45f1d2a 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java
@@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.TenantManageLocator;
import org.openqa.selenium.WebDriver;
public class TenantManagePage extends PageCommon {
+ TenantManageData tenantManageData = new TenantManageData();
+
/**
* Unique constructor
* @param driver driver
@@ -37,7 +39,7 @@ public class TenantManagePage extends PageCommon {
*/
public boolean jumpSecurity() throws InterruptedException {
clickTopElement(TenantManageLocator.SECURITY_CENTER);
- return ifTitleContains(TenantManageData.TENANT_MANAGE);
+ return ifTitleContains(tenantManageData.getTenantData("tenantTitle"));
}
/**
@@ -52,16 +54,16 @@ public class TenantManagePage extends PageCommon {
clickButton(TenantManageLocator.CREATE_TENANT_BUTTON);
// tenant data
- sendInput(TenantManageLocator.TENANT_INPUT_CODE, TenantManageData.TENANT_CODE);
- sendInput(TenantManageLocator.TENANT_INPUT_NAME, TenantManageData.TENANT_NAME);
- sendInput(TenantManageLocator.QUEUE, TenantManageData.QUEUE);
- sendInput(TenantManageLocator.DESCRIPTION, TenantManageData.DESCRIPTION);
+ sendInput(TenantManageLocator.TENANT_INPUT_CODE, tenantManageData.getTenantData("tenantCode"));
+ sendInput(TenantManageLocator.TENANT_INPUT_NAME, tenantManageData.getTenantData("tenantName"));
+ sendInput(TenantManageLocator.QUEUE, tenantManageData.getTenantData("queue"));
+ sendInput(TenantManageLocator.DESCRIPTION, tenantManageData.getTenantData("description"));
// click button
clickButton(TenantManageLocator.SUBMIT_BUTTON);
// Whether to enter the specified page after submit
- return ifTextExists(TenantManageLocator.TENANT_CODE_FIRST, TenantManageData.TENANT_CODE);
+ return ifTextExists(TenantManageLocator.TENANT_CODE_FIRST, tenantManageData.getTenantData("tenantCode"));
}
public boolean deleteTenant() throws InterruptedException {
@@ -74,6 +76,6 @@ public class TenantManagePage extends PageCommon {
clickButton(TenantManageLocator.CONFIRM_DELETE_TENANT_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(TenantManageData.TENANT_MANAGE);
+ return ifTitleContains(tenantManageData.getTenantData("tenantTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TokenManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TokenManagePage.java
index 61fb93c570..35a9377578 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TokenManagePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TokenManagePage.java
@@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.TokenManageLocator;
import org.openqa.selenium.WebDriver;
public class TokenManagePage extends PageCommon {
+ TokenManageData tokenManageData = new TokenManageData();
+
public TokenManagePage(WebDriver driver) {
super(driver);
}
@@ -36,11 +38,13 @@ public class TokenManagePage extends PageCommon {
clickElement(TokenManageLocator.CLICK_TOKEN_MANAGE);
//determine whether the create token button exists
- ifTextExists(TokenManageLocator.CLICK_CREATE_TOKEN,TokenManageData.CREATE_TOKEN);
+ ifTextExists(TokenManageLocator.CLICK_CREATE_TOKEN, tokenManageData.getTokenData("createTokenText"));
// click create token button
clickButton(TokenManageLocator.CLICK_CREATE_TOKEN);
+ ifTextExists(TokenManageLocator.CREATE_TOKEN_POPUP,tokenManageData.getTokenData("createTokenText"));
+
clickButton(TokenManageLocator.SELECT_USER);
clickButton(TokenManageLocator.CLICK_GENERATE_TOKEN_BUTTON);
@@ -50,7 +54,7 @@ public class TokenManagePage extends PageCommon {
clickButton(TokenManageLocator.CLICK_SUBMIT_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(TokenManageData.TOKEN_MANAGE);
+ return ifTitleContains(tokenManageData.getTokenData("tokenTitle"));
}
/**
@@ -66,7 +70,7 @@ public class TokenManagePage extends PageCommon {
locateElement(TokenManageLocator.EDIT_TOKEN_BUTTON);
// click edit token button
- clickButton(TokenManageLocator.CLICK_EDIT_BUTTON);
+ clickButton(TokenManageLocator.EDIT_TOKEN_BUTTON);
clickButton(TokenManageLocator.SELECT_USER);
@@ -77,7 +81,7 @@ public class TokenManagePage extends PageCommon {
clickButton(TokenManageLocator.CLICK_SUBMIT_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(TokenManageData.TOKEN_MANAGE);
+ return ifTitleContains(tokenManageData.getTokenData("tokenTitle"));
}
@@ -89,7 +93,7 @@ public class TokenManagePage extends PageCommon {
clickButton(TokenManageLocator.CLICK_DELETE_BUTTON);
clickButton(TokenManageLocator.CLICK_CONFIRM_DELETE_BUTTON);
- return ifTitleContains(TokenManageData.TOKEN_MANAGE);
+ return ifTitleContains(tokenManageData.getTokenData("tokenTitle"));
}
}
diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java
index 367896a3cf..b489d12621 100644
--- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java
+++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java
@@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.UserManageLocator;
import org.openqa.selenium.WebDriver;
public class UserManagePage extends PageCommon {
+ UserManageData userManageData = new UserManageData();
+
public UserManagePage(WebDriver driver) {
super(driver);
}
@@ -34,22 +36,21 @@ public class UserManagePage extends PageCommon {
public boolean createUser() throws InterruptedException {
// click user manage
clickElement(UserManageLocator.CLICK_USER_MANAGE);
-
//determine whether the create user button exists
- ifTextExists(UserManageLocator.CLICK_CREATE_USER_BUTTON,UserManageData.CREATE_USER_BUTTON);
+ ifTextExists(UserManageLocator.CLICK_CREATE_USER_BUTTON, userManageData.getUserData("createUserButton"));
// click create user button
clickButton(UserManageLocator.CLICK_CREATE_USER_BUTTON);
// input user data
- sendInput(UserManageLocator.INPUT_USERNAME, UserManageData.USERNAME);
- sendInput(UserManageLocator.INPUT_PASSWORD, UserManageData.PASSWORD);
+ sendInput(UserManageLocator.INPUT_USERNAME, userManageData.getUserData("userName"));
+ sendInput(UserManageLocator.INPUT_PASSWORD, userManageData.getUserData("password"));
clickButton(UserManageLocator.CLICK_TENANT);
clickButton(UserManageLocator.SELECT_TENANT);
clickButton(UserManageLocator.CLICK_QUEUE);
clickButton(UserManageLocator.SELECT_QUEUE);
- sendInput(UserManageLocator.INPUT_EMAIL, UserManageData.EMAIL);
- sendInput(UserManageLocator.INPUT_PHONE, UserManageData.PHONE);
+ sendInput(UserManageLocator.INPUT_EMAIL, userManageData.getUserData("email"));
+ sendInput(UserManageLocator.INPUT_PHONE, userManageData.getUserData("phone"));
clickElement(UserManageLocator.SELECT_STOP_STATE);
clickElement(UserManageLocator.SELECT_ENABLE_STATE);
@@ -57,7 +58,7 @@ public class UserManagePage extends PageCommon {
clickButton(UserManageLocator.SUBMIT);
// Whether to enter the specified page after submit
- return ifTitleContains(UserManageData.USER_MANAGE);
+ return ifTextExists(UserManageLocator.USERNAME, userManageData.getUserData("userName"));
}
/**
@@ -70,14 +71,14 @@ public class UserManagePage extends PageCommon {
clickElement(UserManageLocator.CLICK_USER_MANAGE);
//determine whether the general user exists
- ifTextExists(UserManageLocator.USER_NAME,UserManageData.USERNAME);
+ ifTextExists(UserManageLocator.USERNAME, userManageData.getUserData("userName"));
// click edit user button
clickButton(UserManageLocator.EDIT_GENERAL_USER_BUTTON );
// input user data
- clearSendInput(UserManageLocator.INPUT_USERNAME, UserManageData.EDIT_USERNAME);
- clearSendInput(UserManageLocator.INPUT_PASSWORD, UserManageData.EDIT_PASSWORD);
+ clearSendInput(UserManageLocator.INPUT_USERNAME, userManageData.getUserData("editUserName"));
+ clearSendInput(UserManageLocator.INPUT_PASSWORD, userManageData.getUserData("editPassword"));
clickButton(UserManageLocator.CLICK_TENANT);
clickButton(UserManageLocator.SELECT_TENANT);
@@ -85,8 +86,8 @@ public class UserManagePage extends PageCommon {
clickButton(UserManageLocator.CLICK_QUEUE);
clickButton(UserManageLocator.SELECT_QUEUE);
- clearSendInput(UserManageLocator.INPUT_EMAIL, UserManageData.EDIT_EMAIL);
- clearSendInput(UserManageLocator.INPUT_PHONE, UserManageData.EDIT_PHONE);
+ clearSendInput(UserManageLocator.INPUT_EMAIL, userManageData.getUserData("editEmail"));
+ clearSendInput(UserManageLocator.INPUT_PHONE, userManageData.getUserData("editPhone"));
clickElement(UserManageLocator.SELECT_STOP_STATE);
clickElement(UserManageLocator.SELECT_ENABLE_STATE);
@@ -96,7 +97,7 @@ public class UserManagePage extends PageCommon {
System.out.println("end edit general user");
// Whether to enter the specified page after submit
- return ifTitleContains(UserManageData.USER_MANAGE);
+ return ifTitleContains(userManageData.getUserData("userTitle"));
}
/**
@@ -109,7 +110,7 @@ public class UserManagePage extends PageCommon {
clickElement(UserManageLocator.CLICK_USER_MANAGE);
//determine whether the general user edit success
- ifTextExists(UserManageLocator.USER_NAME,UserManageData.EDIT_USERNAME);
+ ifTextExists(UserManageLocator.USER_NAME,userManageData.getUserData("editUserName"));
// click edit user button
clickButton(UserManageLocator.EDIT_ADMIN_USER_BUTTON );
@@ -123,7 +124,7 @@ public class UserManagePage extends PageCommon {
clickButton(UserManageLocator.SUBMIT);
System.out.println("end edit admin user");
// Whether to enter the specified page after submit
- return ifTitleContains(UserManageData.USER_MANAGE);
+ return ifTitleContains(userManageData.getUserData("userTitle"));
}
@@ -144,6 +145,6 @@ public class UserManagePage extends PageCommon {
clickButton(UserManageLocator.CONFIRM_DELETE_USER_BUTTON);
// Whether to enter the specified page after submit
- return ifTitleContains(UserManageData.USER_MANAGE);
+ return ifTitleContains(userManageData.getUserData("userTitle"));
}
}
diff --git a/e2e/src/test/resources/testData/security_zh_cn.yml b/e2e/src/test/resources/testData/security_zh_cn.yml
new file mode 100644
index 0000000000..89be444df5
--- /dev/null
+++ b/e2e/src/test/resources/testData/security_zh_cn.yml
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+userManage:
+ userName: selenium_user_11111
+ password: 123456qwe
+ email: 123456789@qq.com
+ phone: '15811112222'
+ createUserButton: 创建用户
+ editUserName: selenium_user_edit
+ editPassword: 123456qwe@asd
+ editEmail: 123456_edit@qq.com
+ editPhone: '15800001111'
+ userTitle: 用户管理 - DolphinScheduler
+
+tenantManage:
+ tenantCode: selenium_tenant_code_1
+ tenantName: selenium_tenant_Name
+ queue: default
+ description: create tenant test
+ tenantTitle: 租户管理 - DolphinScheduler
+
+alertManage:
+ alertName: selenium_alert_Name
+ createAlert: 创建告警组
+ alertType: 邮件
+ description: create alert test
+ alertTitle: 告警组管理 - DolphinScheduler
+
+queueManage:
+ queueName: selenium_queue_name
+ queueValue: selenium_queue_value
+ createQueueButton: 创建队列
+ editQueueName: selenium_queue_value_edit
+ editQueueValue: selenium_queue_value_edit
+ queueTitle: 队列管理 - DolphinScheduler
+
+tokenManage:
+ tokenTitle: 令牌管理 - DolphinScheduler
+ createTokenText: 创建令牌
+ userName: admin
diff --git a/e2e/src/test/resources/testData/workflow_zh_cn.yml b/e2e/src/test/resources/testData/workflow_zh_cn.yml
new file mode 100644
index 0000000000..bb6dea7851
--- /dev/null
+++ b/e2e/src/test/resources/testData/workflow_zh_cn.yml
@@ -0,0 +1,63 @@
+#
+# 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.
+#
+
+projectManage:
+ createProjectButton: 创建项目
+ projectName: selenium_project_1
+ description: test create project description
+ projectTitle: 项目 - DolphinScheduler
+
+workflowDefine:
+ workflowDefine: 工作流定义
+ shellTaskName: shell_task_selenium_1
+ shellTaskDescription: shell task description test
+ taskTimeout: '60'
+ shellScript: echo "shell task test"
+ customParameter1: selenium_parameter
+ customParameterValue1: selenium_parameter_123
+ customParameter2: selenium_parameter_delete
+ customParameterValue2: selenium_parameter_delete_456
+ workflowDefineTitle: 工作流定义 - DolphinScheduler
+ createWorkflowTitle: 创建流程定义 - DolphinScheduler
+ workflowName: selenium_shell_1
+ workflowDescription: test selenium_shell_1 description
+ workflowTimeout: '30'
+ globalParameter1: selenium_global_parameters_1
+ globalParameterValue1: selenium_global_parameters_value_1
+ globalParameter2: selenium_global_parameters_2
+ globalParameterValue2: selenium_global_parameters_value_2
+ online: 上线
+ offline: 下线
+
+runWorkflow:
+ recipient: 123456789@qq.com
+ Cc: qwe12312sds@qq.com
+ online: 上线
+ offline: 下线
+
+timing:
+ recipient: test123456@qq.com
+ Cc: test.123qwe@qq.com
+ editRecipient: test.edit123456@qq.com
+ editCc: test.edit123qwe@qq.com
+ timingTitle: 定时任务列表 - DolphinScheduler
+ online: 上线
+ offline: 下线
+
+processInstance:
+ processInstanceTitle: 工作流实例 - DolphinScheduler
+ rerun: 重跑
diff --git a/e2e/testng.xml b/e2e/testng.xml
index e08c4da917..8a4791fea6 100644
--- a/e2e/testng.xml
+++ b/e2e/testng.xml
@@ -56,9 +56,9 @@
+
-
diff --git a/pom.xml b/pom.xml
index 0321c1a24c..f7609611f4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,8 +59,8 @@
UTF-8
UTF-8
4.3.0
- 5.1.5.RELEASE
- 2.1.3.RELEASE
+ 5.1.18.RELEASE
+ 2.1.17.RELEASE
1.8
1.2.3
2.7.3
@@ -69,9 +69,9 @@
3.2.0
2.0.1
5.0.5
- 1.1.14
+ 1.1.22
1.4.200
- 1.6
+ 1.11
1.1.1
4.4.1
4.4.1
@@ -785,6 +785,7 @@
**/common/utils/StringTest.java
**/common/utils/StringUtilsTest.java
**/common/utils/TaskParametersUtilsTest.java
+ **/common/utils/VarPoolUtilsTest.java
**/common/utils/HadoopUtilsTest.java
**/common/utils/HttpUtilsTest.java
**/common/utils/KerberosHttpClientTest.java
@@ -832,6 +833,7 @@
**/server/register/ZookeeperNodeManagerTest.java
**/server/utils/DataxUtilsTest.java
**/server/utils/ExecutionContextTestUtils.java
+ **/server/utils/HostTest.java
**/server/utils/LogUtilsTest.java
**/server/utils/ParamUtilsTest.java
@@ -847,9 +849,11 @@
**/server/worker/task/sqoop/SqoopTaskTest.java
+ **/server/worker/task/TaskManagerTest.java
**/server/worker/EnvFileTest.java
**/server/worker/runner/TaskExecuteThreadTest.java
**/service/quartz/cron/CronUtilsTest.java
+ **/service/process/ProcessServiceTest.java
**/service/zk/DefaultEnsembleProviderTest.java
**/service/zk/ZKServerTest.java
**/service/zk/CuratorZookeeperClientTest.java
diff --git a/sql/dolphinscheduler-postgre.sql b/sql/dolphinscheduler-postgre.sql
index 1fedf05cfd..e2f5ebd91f 100644
--- a/sql/dolphinscheduler-postgre.sql
+++ b/sql/dolphinscheduler-postgre.sql
@@ -377,6 +377,7 @@ CREATE TABLE t_ds_process_instance (
worker_group varchar(64) ,
timeout int DEFAULT '0' ,
tenant_id int NOT NULL DEFAULT '-1' ,
+ var_pool text ,
PRIMARY KEY (id)
) ;
create index process_instance_index on t_ds_process_instance (process_definition_id,id);
@@ -523,7 +524,8 @@ CREATE TABLE t_ds_resources (
pid int,
full_name varchar(64),
is_directory int,
- PRIMARY KEY (id)
+ PRIMARY KEY (id),
+ CONSTRAINT t_ds_resources_un UNIQUE (full_name, type)
) ;
@@ -594,6 +596,7 @@ CREATE TABLE t_ds_task_instance (
executor_id int DEFAULT NULL ,
first_submit_time timestamp DEFAULT NULL ,
delay_time int DEFAULT '0' ,
+ var_pool text ,
PRIMARY KEY (id)
) ;
diff --git a/sql/dolphinscheduler_mysql.sql b/sql/dolphinscheduler_mysql.sql
index 70bb7cddfc..9039a19084 100644
--- a/sql/dolphinscheduler_mysql.sql
+++ b/sql/dolphinscheduler_mysql.sql
@@ -487,6 +487,7 @@ CREATE TABLE `t_ds_process_instance` (
`worker_group` varchar(64) DEFAULT NULL COMMENT 'worker group id',
`timeout` int(11) DEFAULT '0' COMMENT 'time out',
`tenant_id` int(11) NOT NULL DEFAULT '-1' COMMENT 'tenant id',
+ `var_pool` longtext COMMENT 'var_pool',
PRIMARY KEY (`id`),
KEY `process_instance_index` (`process_definition_id`,`id`) USING BTREE,
KEY `start_time_index` (`start_time`) USING BTREE
@@ -657,7 +658,8 @@ CREATE TABLE `t_ds_resources` (
`pid` int(11) DEFAULT NULL,
`full_name` varchar(64) DEFAULT NULL,
`is_directory` tinyint(4) DEFAULT NULL,
- PRIMARY KEY (`id`)
+ PRIMARY KEY (`id`),
+ UNIQUE KEY `t_ds_resources_un` (`full_name`,`type`)
) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8;
-- ----------------------------
@@ -736,6 +738,7 @@ CREATE TABLE `t_ds_task_instance` (
`executor_id` int(11) DEFAULT NULL,
`first_submit_time` datetime DEFAULT NULL COMMENT 'task first submit time',
`delay_time` int(4) DEFAULT '0' COMMENT 'task delay execution time',
+ `var_pool` longtext COMMENT 'var_pool',
PRIMARY KEY (`id`),
KEY `process_instance_id` (`process_instance_id`) USING BTREE,
KEY `task_instance_index` (`process_definition_id`,`process_instance_id`) USING BTREE,
diff --git a/sql/upgrade/1.3.3_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.3.3_schema/mysql/dolphinscheduler_ddl.sql
index e9f8b5b6b5..f7b3bbcabb 100644
--- a/sql/upgrade/1.3.3_schema/mysql/dolphinscheduler_ddl.sql
+++ b/sql/upgrade/1.3.3_schema/mysql/dolphinscheduler_ddl.sql
@@ -56,12 +56,52 @@ delimiter ;
CALL uc_dolphin_T_t_ds_task_instance_A_delay_time();
DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_A_delay_time;
+-- uc_dolphin_T_t_ds_task_instance_A_var_pool
+drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_task_instance_A_var_pool;
+delimiter d//
+CREATE PROCEDURE uc_dolphin_T_t_ds_task_instance_A_var_pool()
+ BEGIN
+ IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_task_instance'
+ AND TABLE_SCHEMA=(SELECT DATABASE())
+ AND COLUMN_NAME ='var_pool')
+ THEN
+ ALTER TABLE t_ds_task_instance ADD `var_pool` longtext NULL;
+ END IF;
+ END;
+
+d//
+
+delimiter ;
+CALL uc_dolphin_T_t_ds_task_instance_A_var_pool();
+DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_A_var_pool;
+
+-- uc_dolphin_T_t_ds_process_instance_A_var_pool
+drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_process_instance_A_var_pool;
+delimiter d//
+CREATE PROCEDURE uc_dolphin_T_t_ds_process_instance_A_var_pool()
+ BEGIN
+ IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_process_instance'
+ AND TABLE_SCHEMA=(SELECT DATABASE())
+ AND COLUMN_NAME ='var_pool')
+ THEN
+ ALTER TABLE t_ds_process_instance ADD `var_pool` longtext NULL;
+ END IF;
+ END;
+
+d//
+
+delimiter ;
+CALL uc_dolphin_T_t_ds_process_instance_A_var_pool();
+DROP PROCEDURE uc_dolphin_T_t_ds_process_instance_A_var_pool;
+
-- uc_dolphin_T_t_ds_process_definition_A_modify_by
drop PROCEDURE if EXISTS ct_dolphin_T_t_ds_process_definition_version;
delimiter d//
CREATE PROCEDURE ct_dolphin_T_t_ds_process_definition_version()
BEGIN
- CREATE TABLE `t_ds_process_definition_version` (
+ CREATE TABLE IF NOT EXISTS `t_ds_process_definition_version` (
`id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'key',
`process_definition_id` int(11) NOT NULL COMMENT 'process definition id',
`version` int(11) DEFAULT NULL COMMENT 'process definition version',
@@ -87,3 +127,29 @@ delimiter ;
CALL ct_dolphin_T_t_ds_process_definition_version;
DROP PROCEDURE ct_dolphin_T_t_ds_process_definition_version;
+
+
+
+-- add t_ds_resources_un
+DROP PROCEDURE IF EXISTS uc_dolphin_T_t_ds_resources_un;
+delimiter d//
+CREATE PROCEDURE uc_dolphin_T_t_ds_resources_un()
+BEGIN
+ IF NOT EXISTS (
+ SELECT * FROM information_schema.KEY_COLUMN_USAGE
+ WHERE TABLE_NAME = 't_ds_resources'
+ AND CONSTRAINT_NAME = 't_ds_resources_un'
+ )
+ THEN
+ ALTER TABLE t_ds_resources ADD CONSTRAINT t_ds_resources_un UNIQUE KEY (full_name,`type`);
+ END IF;
+END;
+
+d//
+
+delimiter ;
+CALL uc_dolphin_T_t_ds_resources_un();
+DROP PROCEDURE IF EXISTS uc_dolphin_T_t_ds_resources_un;
+
+
+
diff --git a/sql/upgrade/1.3.3_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.3.3_schema/postgresql/dolphinscheduler_ddl.sql
index 52045f61f0..477cb3bf60 100644
--- a/sql/upgrade/1.3.3_schema/postgresql/dolphinscheduler_ddl.sql
+++ b/sql/upgrade/1.3.3_schema/postgresql/dolphinscheduler_ddl.sql
@@ -51,11 +51,47 @@ delimiter ;
SELECT uc_dolphin_T_t_ds_task_instance_A_delay_time();
DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_A_delay_time();
+-- uc_dolphin_T_t_ds_process_instance_A_var_pool
+delimiter d//
+CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_process_instance_A_var_pool() RETURNS void AS $$
+BEGIN
+ IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_process_instance'
+ AND COLUMN_NAME ='var_pool')
+ THEN
+ ALTER TABLE t_ds_process_instance ADD COLUMN var_pool text;
+ END IF;
+END;
+$$ LANGUAGE plpgsql;
+d//
+
+delimiter ;
+SELECT uc_dolphin_T_t_ds_process_instance_A_var_pool();
+DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_process_instance_A_var_pool();
+
+-- uc_dolphin_T_t_ds_task_instance_A_var_pool
+delimiter d//
+CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_task_instance_A_var_pool() RETURNS void AS $$
+BEGIN
+ IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_task_instance'
+ AND COLUMN_NAME ='var_pool')
+ THEN
+ ALTER TABLE t_ds_task_instance ADD COLUMN var_pool text;
+ END IF;
+END;
+$$ LANGUAGE plpgsql;
+d//
+
+delimiter ;
+SELECT uc_dolphin_T_t_ds_task_instance_A_var_pool();
+DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_A_var_pool();
+
-- uc_dolphin_T_t_ds_process_definition_A_modify_by
delimiter d//
CREATE OR REPLACE FUNCTION ct_dolphin_T_t_ds_process_definition_version() RETURNS void AS $$
BEGIN
-CREATE TABLE t_ds_process_definition_version (
+CREATE TABLE IF NOT EXISTS t_ds_process_definition_version (
id int NOT NULL ,
process_definition_id int NOT NULL ,
version int DEFAULT NULL ,
@@ -79,4 +115,28 @@ d//
delimiter ;
SELECT ct_dolphin_T_t_ds_process_definition_version();
-DROP FUNCTION IF EXISTS ct_dolphin_T_t_ds_process_definition_version();
\ No newline at end of file
+DROP FUNCTION IF EXISTS ct_dolphin_T_t_ds_process_definition_version();
+
+
+
+
+-- add t_ds_resources_un
+CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_resources_un() RETURNS void AS $$
+BEGIN
+ IF NOT EXISTS (
+ SELECT 1 FROM information_schema.KEY_COLUMN_USAGE
+ WHERE TABLE_NAME = 't_ds_resources'
+ AND CONSTRAINT_NAME = 't_ds_resources_un'
+ )
+ THEN
+ALTER TABLE t_ds_resources ADD CONSTRAINT t_ds_resources_un UNIQUE (full_name,"type");
+END IF;
+END;
+$$ LANGUAGE plpgsql;
+
+SELECT uc_dolphin_T_t_ds_resources_un();
+DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_resources_un();
+
+
+
+
diff --git a/tools/dependencies/check-LICENSE.sh b/tools/dependencies/check-LICENSE.sh
index d414bd40c6..0072554e29 100755
--- a/tools/dependencies/check-LICENSE.sh
+++ b/tools/dependencies/check-LICENSE.sh
@@ -25,7 +25,9 @@ tar -zxf dolphinscheduler-dist/target/apache-dolphinscheduler*-bin.tar.gz --stri
# licenses
echo '=== Self modules: ' && ./mvnw --batch-mode --quiet -Dexec.executable='echo' -Dexec.args='${project.artifactId}-${project.version}.jar' exec:exec | tee self-modules.txt
-echo '=== Distributed dependencies: ' && ls dist/lib | tee all-dependencies.txt
+echo '=== Distributed dependencies: ' && find dist/lib -name "*.jar" | tee all-dependencies.txt
+# The prefix "dist/lib/" (9 chars) should be stripped to be ready to compare
+sed -i 's/.\{9\}//' all-dependencies.txt
# Exclude all self modules(jars) to generate all third-party dependencies
echo '=== Third party dependencies: ' && grep -vf self-modules.txt all-dependencies.txt | tee third-party-dependencies.txt
diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt
index 0074dcd9fb..96089d5dcd 100755
--- a/tools/dependencies/known-dependencies.txt
+++ b/tools/dependencies/known-dependencies.txt
@@ -2,26 +2,26 @@ HikariCP-3.2.0.jar
activation-1.1.jar
ant-1.6.5.jar
aopalliance-1.0.jar
-apache-el-8.5.35.1.jar
+apache-el-8.5.54.jar
apacheds-i18n-2.0.0-M15.jar
apacheds-kerberos-codec-2.0.0-M15.jar
api-asn1-api-1.0.0-M20.jar
api-util-1.0.0-M20.jar
asm-3.1.jar
-aspectjweaver-1.9.2.jar
+aspectjweaver-1.9.6.jar
audience-annotations-0.5.0.jar
avro-1.7.4.jar
aws-java-sdk-1.7.4.jar
bonecp-0.8.0.RELEASE.jar
-byte-buddy-1.9.10.jar
+byte-buddy-1.9.16.jar
classmate-1.4.0.jar
clickhouse-jdbc-0.1.52.jar
commons-cli-1.2.jar
-commons-codec-1.6.jar
+commons-codec-1.11.jar
commons-collections-3.2.2.jar
commons-collections4-4.1.jar
commons-compress-1.4.1.jar
-commons-compiler-3.0.12.jar
+commons-compiler-3.0.16.jar
commons-configuration-1.10.jar
commons-daemon-1.0.13.jar
commons-beanutils-1.7.0.jar
@@ -43,8 +43,8 @@ datanucleus-api-jdo-4.2.1.jar
datanucleus-core-4.1.6.jar
datanucleus-rdbms-4.1.7.jar
derby-10.14.2.0.jar
-druid-1.1.14.jar
-gson-2.8.5.jar
+druid-1.1.22.jar
+gson-2.8.6.jar
guava-20.0.jar
guice-3.0.jar
guice-servlet-3.0.jar
@@ -65,7 +65,7 @@ hadoop-yarn-client-2.7.3.jar
hadoop-yarn-common-2.7.3.jar
hadoop-yarn-server-common-2.7.3.jar
hamcrest-core-1.3.jar
-hibernate-validator-6.0.14.Final.jar
+hibernate-validator-6.0.20.Final.jar
hive-common-2.1.0.jar
hive-jdbc-2.1.0.jar
hive-metastore-2.1.0.jar
@@ -77,19 +77,19 @@ hive-storage-api-2.1.0.jar
htrace-core-3.1.0-incubating.jar
httpclient-4.4.1.jar
httpcore-4.4.1.jar
-httpmime-4.5.7.jar
+httpmime-4.5.12.jar
jackson-annotations-2.9.8.jar
jackson-core-2.9.8.jar
jackson-core-asl-1.9.13.jar
jackson-databind-2.9.8.jar
-jackson-datatype-jdk8-2.9.8.jar
-jackson-datatype-jsr310-2.9.8.jar
+jackson-datatype-jdk8-2.9.10.jar
+jackson-datatype-jsr310-2.9.10.jar
jackson-jaxrs-1.9.13.jar
jackson-mapper-asl-1.9.13.jar
-jackson-module-parameter-names-2.9.8.jar
+jackson-module-parameter-names-2.9.10.jar
jackson-xc-1.9.13.jar
jamon-runtime-2.3.1.jar
-janino-3.0.12.jar
+janino-3.0.16.jar
java-xmlbuilder-0.4.jar
javax.activation-api-1.2.0.jar
javax.annotation-api-1.3.2.jar
@@ -100,7 +100,7 @@ javax.servlet-api-3.1.0.jar
javolution-5.5.1.jar
jaxb-api-2.3.1.jar
jaxb-impl-2.2.3-1.jar
-jboss-logging-3.3.2.Final.jar
+jboss-logging-3.3.3.Final.jar
jdo-api-3.0.1.jar
jersey-client-1.9.jar
jersey-core-1.9.jar
@@ -110,21 +110,21 @@ jersey-server-1.9.jar
jets3t-0.9.0.jar
jettison-1.1.jar
jetty-6.1.26.jar
-jetty-continuation-9.4.14.v20181114.jar
-jetty-http-9.4.14.v20181114.jar
-jetty-io-9.4.14.v20181114.jar
-jetty-security-9.4.14.v20181114.jar
-jetty-server-9.4.14.v20181114.jar
-jetty-servlet-9.4.14.v20181114.jar
-jetty-servlets-9.4.14.v20181114.jar
+jetty-continuation-9.4.31.v20200723.jar
+jetty-http-9.4.31.v20200723.jar
+jetty-io-9.4.31.v20200723.jar
+jetty-security-9.4.31.v20200723.jar
+jetty-server-9.4.31.v20200723.jar
+jetty-servlet-9.4.31.v20200723.jar
+jetty-servlets-9.4.31.v20200723.jar
jetty-util-6.1.26.jar
-jetty-util-9.4.14.v20181114.jar
-jetty-webapp-9.4.14.v20181114.jar
-jetty-xml-9.4.14.v20181114.jar
+jetty-util-9.4.31.v20200723.jar
+jetty-webapp-9.4.31.v20200723.jar
+jetty-xml-9.4.31.v20200723.jar
jline-0.9.94.jar
jna-4.5.2.jar
jna-platform-4.5.2.jar
-joda-time-2.10.1.jar
+joda-time-2.10.6.jar
jpam-1.1.jar
jsch-0.1.42.jar
jsp-2.1-6.1.14.jar
@@ -133,7 +133,7 @@ jsp-api-2.1.jar
jsqlparser-2.1.jar
jsr305-3.0.0.jar
jta-1.1.jar
-jul-to-slf4j-1.7.25.jar
+jul-to-slf4j-1.7.30.jar
junit-4.12.jar
leveldbjni-all-1.8.jar
libfb303-0.9.3.jar
@@ -155,7 +155,7 @@ mybatis-plus-core-3.2.0.jar
mybatis-plus-extension-3.2.0.jar
mybatis-spring-2.0.2.jar
netty-3.6.2.Final.jar
-netty-all-4.1.33.Final.jar
+netty-all-4.1.52.Final.jar
opencsv-2.3.jar
oshi-core-3.5.0.jar
paranamer-2.3.jar
@@ -169,27 +169,27 @@ slf4j-api-1.7.5.jar
snakeyaml-1.23.jar
snappy-0.2.jar
snappy-java-1.0.4.1.jar
-spring-aop-5.1.5.RELEASE.jar
-spring-beans-5.1.5.RELEASE.jar
-spring-boot-2.1.3.RELEASE.jar
-spring-boot-autoconfigure-2.1.3.RELEASE.jar
-spring-boot-starter-2.1.3.RELEASE.jar
-spring-boot-starter-aop-2.1.3.RELEASE.jar
-spring-boot-starter-jdbc-2.1.3.RELEASE.jar
-spring-boot-starter-jetty-2.1.3.RELEASE.jar
-spring-boot-starter-json-2.1.3.RELEASE.jar
-spring-boot-starter-logging-2.1.3.RELEASE.jar
-spring-boot-starter-web-2.1.3.RELEASE.jar
-spring-context-5.1.5.RELEASE.jar
-spring-core-5.1.5.RELEASE.jar
-spring-expression-5.1.5.RELEASE.jar
-spring-jcl-5.1.5.RELEASE.jar
-spring-jdbc-5.1.5.RELEASE.jar
+spring-aop-5.1.18.RELEASE.jar
+spring-beans-5.1.18.RELEASE.jar
+spring-boot-2.1.17.RELEASE.jar
+spring-boot-autoconfigure-2.1.17.RELEASE.jar
+spring-boot-starter-2.1.17.RELEASE.jar
+spring-boot-starter-aop-2.1.17.RELEASE.jar
+spring-boot-starter-jdbc-2.1.17.RELEASE.jar
+spring-boot-starter-jetty-2.1.17.RELEASE.jar
+spring-boot-starter-json-2.1.17.RELEASE.jar
+spring-boot-starter-logging-2.1.17.RELEASE.jar
+spring-boot-starter-web-2.1.17.RELEASE.jar
+spring-context-5.1.18.RELEASE.jar
+spring-core-5.1.18.RELEASE.jar
+spring-expression-5.1.18.RELEASE.jar
+spring-jcl-5.1.18.RELEASE.jar
+spring-jdbc-5.1.18.RELEASE.jar
spring-plugin-core-1.2.0.RELEASE.jar
spring-plugin-metadata-1.2.0.RELEASE.jar
-spring-tx-5.1.5.RELEASE.jar
-spring-web-5.1.5.RELEASE.jar
-spring-webmvc-5.1.5.RELEASE.jar
+spring-tx-5.1.18.RELEASE.jar
+spring-web-5.1.18.RELEASE.jar
+spring-webmvc-5.1.18.RELEASE.jar
springfox-core-2.9.2.jar
springfox-schema-2.9.2.jar
springfox-spi-2.9.2.jar
@@ -210,4 +210,4 @@ xmlenc-0.52.jar
xz-1.0.jar
zookeeper-3.4.14.jar
guava-retrying-2.0.0.jar
-presto-jdbc-0.238.1.jar
\ No newline at end of file
+presto-jdbc-0.238.1.jar
|