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

496 lines
20 KiB

/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.utils;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.graph.DAG;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
import org.apache.dolphinscheduler.common.process.ProcessDag;
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessData;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
5 years ago
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* dag tools
*/
public class DagHelper {
private static final Logger logger = LoggerFactory.getLogger(DagHelper.class);
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
/**
* generate flow node relation list by task node list;
* Edges that are not in the task Node List will not be added to the result
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param taskNodeList taskNodeList
* @return task node relation list
*/
fix bug: task cannot submit when recovery failover (#1011) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert
5 years ago
public static List<TaskNodeRelation> generateRelationListByFlowNodes(List<TaskNode> taskNodeList) {
List<TaskNodeRelation> nodeRelationList = new ArrayList<>();
for (TaskNode taskNode : taskNodeList) {
String preTasks = taskNode.getPreTasks();
List<String> preTaskList = JSONUtils.toList(preTasks, String.class);
if (preTaskList != null) {
for (String depNodeName : preTaskList) {
if (null != findNodeByName(taskNodeList, depNodeName)) {
nodeRelationList.add(new TaskNodeRelation(depNodeName, taskNode.getName()));
}
}
}
}
return nodeRelationList;
}
/**
* generate task nodes needed by dag
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param taskNodeList taskNodeList
* @param startNodeNameList startNodeNameList
* @param recoveryNodeNameList recoveryNodeNameList
* @param taskDependType taskDependType
* @return task node list
*/
fix bug: task cannot submit when recovery failover (#1011) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert
5 years ago
public static List<TaskNode> generateFlowNodeListByStartNode(List<TaskNode> taskNodeList, List<String> startNodeNameList,
List<String> recoveryNodeNameList, TaskDependType taskDependType) {
List<TaskNode> destFlowNodeList = new ArrayList<>();
List<String> startNodeList = startNodeNameList;
if(taskDependType != TaskDependType.TASK_POST
&& CollectionUtils.isEmpty(startNodeList)){
logger.error("start node list is empty! cannot continue run the process ");
return destFlowNodeList;
}
List<TaskNode> destTaskNodeList = new ArrayList<>();
List<TaskNode> tmpTaskNodeList = new ArrayList<>();
if (taskDependType == TaskDependType.TASK_POST
&& CollectionUtils.isNotEmpty(recoveryNodeNameList)) {
startNodeList = recoveryNodeNameList;
}
if (CollectionUtils.isEmpty(startNodeList)) {
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
// no special designation start nodes
tmpTaskNodeList = taskNodeList;
} else {
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
// specified start nodes or resume execution
for (String startNodeName : startNodeList) {
TaskNode startNode = findNodeByName(taskNodeList, startNodeName);
List<TaskNode> childNodeList = new ArrayList<>();
if (startNode == null) {
logger.error("start node name [{}] is not in task node list [{}] ",
startNodeName,
taskNodeList
);
continue;
} else if (TaskDependType.TASK_POST == taskDependType) {
List<String> visitedNodeNameList = new ArrayList<>();
childNodeList = getFlowNodeListPost(startNode, taskNodeList, visitedNodeNameList);
} else if (TaskDependType.TASK_PRE == taskDependType) {
List<String> visitedNodeNameList = new ArrayList<>();
childNodeList = getFlowNodeListPre(startNode, recoveryNodeNameList, taskNodeList, visitedNodeNameList);
} else {
childNodeList.add(startNode);
}
tmpTaskNodeList.addAll(childNodeList);
}
}
for (TaskNode taskNode : tmpTaskNodeList) {
if (null == findNodeByName(destTaskNodeList, taskNode.getName())) {
destTaskNodeList.add(taskNode);
}
}
return destTaskNodeList;
}
/**
* find all the nodes that depended on the start node
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param startNode startNode
* @param taskNodeList taskNodeList
* @return task node list
*/
private static List<TaskNode> getFlowNodeListPost(TaskNode startNode, List<TaskNode> taskNodeList, List<String> visitedNodeNameList) {
List<TaskNode> resultList = new ArrayList<>();
for (TaskNode taskNode : taskNodeList) {
List<String> depList = taskNode.getDepList();
if (null != depList && null != startNode && depList.contains(startNode.getName()) && !visitedNodeNameList.contains(taskNode.getName())) {
resultList.addAll(getFlowNodeListPost(taskNode, taskNodeList, visitedNodeNameList));
}
}
// why add (startNode != null) condition? for SonarCloud Quality Gate passed
if (null != startNode) {
visitedNodeNameList.add(startNode.getName());
}
resultList.add(startNode);
return resultList;
}
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
/**
* find all nodes that start nodes depend on.
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param startNode startNode
* @param recoveryNodeNameList recoveryNodeNameList
* @param taskNodeList taskNodeList
* @return task node list
*/
private static List<TaskNode> getFlowNodeListPre(TaskNode startNode, List<String> recoveryNodeNameList, List<TaskNode> taskNodeList, List<String> visitedNodeNameList) {
List<TaskNode> resultList = new ArrayList<>();
List<String> depList = new ArrayList<>();
if (null != startNode) {
depList = startNode.getDepList();
resultList.add(startNode);
}
if (CollectionUtils.isEmpty(depList)) {
return resultList;
}
for (String depNodeName : depList) {
TaskNode start = findNodeByName(taskNodeList, depNodeName);
if (recoveryNodeNameList.contains(depNodeName)) {
resultList.add(start);
} else if (!visitedNodeNameList.contains(depNodeName)) {
resultList.addAll(getFlowNodeListPre(start, recoveryNodeNameList, taskNodeList, visitedNodeNameList));
}
}
// why add (startNode != null) condition? for SonarCloud Quality Gate passed
if (null != startNode) {
visitedNodeNameList.add(startNode.getName());
}
return resultList;
}
/**
* generate dag by start nodes and recovery nodes
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param processDefinitionJson processDefinitionJson
* @param startNodeNameList startNodeNameList
* @param recoveryNodeNameList recoveryNodeNameList
* @param depNodeType depNodeType
* @return process dag
* @throws Exception if error throws Exception
*/
public static ProcessDag generateFlowDag(String processDefinitionJson,
List<String> startNodeNameList,
List<String> recoveryNodeNameList,
TaskDependType depNodeType) throws Exception {
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
List<TaskNode> taskNodeList = new ArrayList<>();
if (null != processData) {
taskNodeList = processData.getTasks();
}
List<TaskNode> destTaskNodeList = generateFlowNodeListByStartNode(taskNodeList, startNodeNameList, recoveryNodeNameList, depNodeType);
if (destTaskNodeList.isEmpty()) {
return null;
}
List<TaskNodeRelation> taskNodeRelations = generateRelationListByFlowNodes(destTaskNodeList);
ProcessDag processDag = new ProcessDag();
processDag.setEdges(taskNodeRelations);
processDag.setNodes(destTaskNodeList);
return processDag;
}
/**
* parse the forbidden task nodes in process definition.
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param processDefinitionJson processDefinitionJson
* @return task node map
*/
public static Map<String, TaskNode> getForbiddenTaskNodeMaps(String processDefinitionJson){
Map<String, TaskNode> forbidTaskNodeMap = new ConcurrentHashMap<>();
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
List<TaskNode> taskNodeList = new ArrayList<>();
if (null != processData) {
taskNodeList = processData.getTasks();
}
for(TaskNode node : taskNodeList){
if(node.isForbidden()){
forbidTaskNodeMap.putIfAbsent(node.getName(), node);
}
}
return forbidTaskNodeMap;
}
/**
* find node by node name
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param nodeDetails nodeDetails
* @param nodeName nodeName
* @return task node
*/
public static TaskNode findNodeByName(List<TaskNode> nodeDetails, String nodeName) {
for (TaskNode taskNode : nodeDetails) {
if (taskNode.getName().equals(nodeName)) {
return taskNode;
}
}
return null;
}
/**
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
* the task can be submit when all the depends nodes are forbidden or complete
* @param taskNode taskNode
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* @param dag dag
* @param completeTaskList completeTaskList
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
* @return can submit
*/
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
public static boolean allDependsForbiddenOrEnd(TaskNode taskNode,
DAG<String, TaskNode, TaskNodeRelation> dag,
Map<String, TaskNode> skipTaskNodeList,
Map<String, TaskInstance> completeTaskList) {
List<String> dependList = taskNode.getDepList();
if (dependList == null) {
return true;
5 years ago
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
for (String dependNodeName : dependList) {
TaskNode dependNode = dag.getNode(dependNodeName);
if (completeTaskList.containsKey(dependNodeName)
|| dependNode.isForbidden()
|| skipTaskNodeList.containsKey(dependNodeName)) {
continue;
} else {
return false;
}
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
return true;
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
/**
* parse the successor nodes of previous node.
* this function parse the condition node to find the right branch.
* also check all the depends nodes forbidden or complete
* @param preNodeName
* @return successor nodes
*/
public static Set<String> parsePostNodes(String preNodeName,
Map<String, TaskNode> skipTaskNodeList,
DAG<String, TaskNode, TaskNodeRelation> dag,
Map<String, TaskInstance> completeTaskList) {
Set<String> postNodeList = new HashSet<>();
Collection<String> startVertexes = new ArrayList<>();
if (preNodeName == null) {
startVertexes = dag.getBeginNode();
} else if (dag.getNode(preNodeName).isConditionsTask()) {
List<String> conditionTaskList = parseConditionTask(preNodeName, skipTaskNodeList, dag, completeTaskList);
startVertexes.addAll(conditionTaskList);
} else {
startVertexes = dag.getSubsequentNodes(preNodeName);
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
for (String subsequent : startVertexes) {
TaskNode taskNode = dag.getNode(subsequent);
if (isTaskNodeNeedSkip(taskNode, skipTaskNodeList)) {
setTaskNodeSkip(subsequent, dag, completeTaskList, skipTaskNodeList );
continue;
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
if (!DagHelper.allDependsForbiddenOrEnd(taskNode, dag, skipTaskNodeList, completeTaskList)) {
continue;
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
if (taskNode.isForbidden() || completeTaskList.containsKey(subsequent)) {
postNodeList.addAll(parsePostNodes(subsequent, skipTaskNodeList, dag, completeTaskList));
continue;
}
postNodeList.add(subsequent);
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
return postNodeList;
}
/**
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
* if all of the task dependence are skipped, skip it too.
* @param taskNode
* @return
*/
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
private static boolean isTaskNodeNeedSkip(TaskNode taskNode,
Map<String, TaskNode> skipTaskNodeList
){
if(CollectionUtils.isEmpty(taskNode.getDepList())){
return false;
fix bug: task cannot submit when recovery failover (#1011) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert
5 years ago
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
for(String depNode : taskNode.getDepList()){
if(!skipTaskNodeList.containsKey(depNode)){
return false;
}
}
return true;
}
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
/**
* parse condition task find the branch process
* set skip flag for another one.
* @param nodeName
* @return
*/
public static List<String> parseConditionTask(String nodeName,
Map<String, TaskNode> skipTaskNodeList,
DAG<String, TaskNode, TaskNodeRelation> dag,
Map<String, TaskInstance> completeTaskList){
List<String> conditionTaskList = new ArrayList<>();
TaskNode taskNode = dag.getNode(nodeName);
if (!taskNode.isConditionsTask()){
return conditionTaskList;
}
if (!completeTaskList.containsKey(nodeName)){
return conditionTaskList;
}
TaskInstance taskInstance = completeTaskList.get(nodeName);
ConditionsParameters conditionsParameters =
JSONUtils.parseObject(taskNode.getConditionResult(), ConditionsParameters.class);
List<String> skipNodeList = new ArrayList<>();
if(taskInstance.getState().typeIsSuccess()){
conditionTaskList = conditionsParameters.getSuccessNode();
skipNodeList = conditionsParameters.getFailedNode();
}else if(taskInstance.getState().typeIsFailure()){
conditionTaskList = conditionsParameters.getFailedNode();
skipNodeList = conditionsParameters.getSuccessNode();
}else{
conditionTaskList.add(nodeName);
}
for(String failedNode : skipNodeList){
setTaskNodeSkip(failedNode, dag, completeTaskList, skipTaskNodeList);
}
return conditionTaskList;
}
/**
* set task node and the post nodes skip flag
* @param skipNodeName
* @param dag
* @param completeTaskList
* @param skipTaskNodeList
*/
private static void setTaskNodeSkip(String skipNodeName,
DAG<String, TaskNode, TaskNodeRelation> dag,
Map<String, TaskInstance> completeTaskList,
Map<String, TaskNode> skipTaskNodeList){
skipTaskNodeList.putIfAbsent(skipNodeName, dag.getNode(skipNodeName));
Collection<String> postNodeList = dag.getSubsequentNodes(skipNodeName);
for(String post : postNodeList){
TaskNode postNode = dag.getNode(post);
if(isTaskNodeNeedSkip(postNode, skipTaskNodeList)){
setTaskNodeSkip(post, dag, completeTaskList, skipTaskNodeList);
}
}
}
/***
dao method comments and useless code modify (#1214) * add ConnectionFactoryTest and ConnectionFactory read datasource from appliction.yml * .escheduler_env.sh to dolphinscheduler_env.sh * dao yml assembly to conf directory * table name modify * entity title table name modify * logback log name modify * running through the big process * running through the big process error modify * logback log name modify * data_source.properties rename * logback log name modify * install.sh optimization * install.sh optimization * command count modify * command state update * countCommandState sql update * countCommandState sql update * remove application.yml file * master.properties modify * install.sh modify * install.sh modify * api server startup modify * the current user quits and the session is completely emptied. bug fix * remove pom package resources * checkQueueNameExist method update * checkQueueExist * install.sh error output update * signOut error update * ProcessDao is null bug fix * install.sh add mail.user * request url variables replace * process define import bug fix * process define import export bug fix * processdefine import export bug fix * down log suffix format modify * import export process define contains crontab error bug fix * add Flink local mode * ProcessDao is null bug fix * loadAverage display problem bug fix * MasterServer rename Server * rollback .env * rollback .env * MasterServer rename Server * the task is abnormal and task is running bug fix * owners and administrators can delete * dockerfile optimization * dockerfile optimization * dockerfile optimization * remove application-alert.properties * task log print worker log bug fix * remove .escheduler_env.sh * change dockerfile email address * dockerfile dao application.properties and install.sh modify * application.properties modify * application.properties modify * dockerfile startup.sh modify * remove docs * nginx conf modify * dockerfile application.properties modify * dockerfile email address change * the alert module is modified in English. * alert server comment and chinese modify * api server useless code and chinese modify * common,dao,server useless code and chinese modify * change chinese to english * dao method comments and useless code modify
5 years ago
* build dag graph
* @param processDag processDag
* @return dag
*/
public static DAG<String, TaskNode, TaskNodeRelation> buildDagGraph(ProcessDag processDag) {
DAG<String,TaskNode,TaskNodeRelation> dag = new DAG<>();
//add vertex
if (CollectionUtils.isNotEmpty(processDag.getNodes())){
for (TaskNode node : processDag.getNodes()){
dag.addNode(node.getName(),node);
}
}
//add edge
if (CollectionUtils.isNotEmpty(processDag.getEdges())){
for (TaskNodeRelation edge : processDag.getEdges()){
dag.addEdge(edge.getStartNode(),edge.getEndNode());
}
}
return dag;
}
/**
* get process dag
* @param taskNodeList task node list
* @return Process dag
*/
public static ProcessDag getProcessDag(List<TaskNode> taskNodeList) {
List<TaskNodeRelation> taskNodeRelations = new ArrayList<>();
// Traverse node information and build relationships
for (TaskNode taskNode : taskNodeList) {
String preTasks = taskNode.getPreTasks();
List<String> preTasksList = JSONUtils.toList(preTasks, String.class);
// If the dependency is not empty
if (preTasksList != null) {
for (String depNode : preTasksList) {
taskNodeRelations.add(new TaskNodeRelation(depNode, taskNode.getName()));
}
}
}
ProcessDag processDag = new ProcessDag();
processDag.setEdges(taskNodeRelations);
processDag.setNodes(taskNodeList);
return processDag;
}
/**
* is there have conditions after the parent node
* @param parentNodeName
* @return
*/
public static boolean haveConditionsAfterNode(String parentNodeName,
DAG<String, TaskNode, TaskNodeRelation> dag
[Draft][Merge][133-dev]133 merge dev (#4031) * [Feture-3327][ui]Add the function of re-uploading files in the resource center * [Feture-3327][ui]Add the function of re-uploading files in the resource center (#3394) * Before creating a workflow, clear the canvas * [Fix-3256][ui] herry pick commit from dev for Fix admin user info update error (#3306) * [Feture-3327][ui]Add the function of re-uploading files in the resource center Co-authored-by: wuchunfu <319355703@qq.com> * [Improvement-3327][api]support re-upload the resource file (#3395) * [Fix-3390][server]Running hive sql task need find the hdfs path correctly (#3396) * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * [Fix-3390][api]Running hive sql task need find the hdfs path correctly * update soft version * hive UDF function to modify the background color * fix * fix bug: Fix master task dependency check bug * cancel spark task version check (#3406) Co-authored-by: Eights-Li <yelli.hl@gmail.com> * [Bug][ui]Fix front-end bug #3413 * [Feature][ambari_plugin]support one worker can belongs different worker groups when execute install script (#3410) * Optimize dag * Update actions.js (#3401) * [Fix-3256][ui] Fix admin user info update error (#3425) (#3428) * [PROPOSAL-3139] Datasource selection changes from radio to select * [PROPOSAL-3139] Datasource selection changes from radio to select * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [BUG FIX] issues #3256 * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * [Fix-3256][ui] Fix admin user info update error * reset createUser.vue * [Fix-3256][ui] Fix admin user info update error Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [Fix-3433][api]Fixed that release the imported process definition which version is below 1.3.0 will be failure * dag connection add check * fix * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x (#3454) * Remove node deep monitoring * If worker group id is null,don't need to set the value of the worker group (#3460) * [Fix-3423][dao][sql]Fixed that the resource file of the task node can't be found when upgrade from 1.2.0 to 1.3.x * [Fix-3423][dao]If worker group id is null,don't need to set the value of the worker group * [ui]Code optimization * fix * fix * [fix-3058][ui]Move rtTargetArr to jsPlumbHandle.js * [optimization][ui]Prevent the shell script input box from being empty * [Fix-3462][api]If login user is admin,need list all udfs (#3465) * [Fix-3462][api]If login user is admin,need list all udfs * [Fix-3462][api]add the test on the method of QueryUdfFuncList * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3462][api]fix the code smell * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource (#3482) * [fixBug-3058][ui]Fix connection abnormalities in historical workflow instance data * [Feture-3327][ui]Add the function of re-uploading files in the udf subdirectory * fix bug: Fix master task dependency check bug (#3473) Co-authored-by: lenboo <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ci_e2e fail (#3497) * [Fix-3469][api]Should filter the resource by the different program type (#3498) * [Fix-3463][api]Fixed that run the sql task will be failure after rename the udf resource * [Fix-3469][api]Should list python file and jar file * [Fix-3469][api]Should filter the resource by the different program type * [Fix-3469][api]fix the code smell * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3499) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * [Fix-3469][ui]The value of maintenance resources and the filtering of resources according to different program types * fix * Revert "fix ci_e2e fail (#3497)" This reverts commit e367f90bb73c9682739308a0a98887a1c0f407ef. * test * test release 1.3.2 version rollback * test release 1.3.2 version rollback (#3503) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * test release 1.3.2 version rollback (#3504) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * fix ds muti-level directory in zk, which lead to fail to assign work * add login user check some actions in api * [Hotfix][ci] Fix e2e ci docker image build error * modify tag 1.3.0 to HEAD * modify tag 1.3.0 to HEAD (#3525) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * remove OGNL part of the mybaits notice (#3526) * [maven-release-plugin] prepare release 1.3.2 * [maven-release-plugin] prepare for next development iteration * release 1.3.2 version rollback (#3527) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Script input box to modify the delay loading time * fix * fix * fix * fix * modify general user can't create token * [ui]It is forbidden to select non-existent resources and modify the tree display data format * modify general user can't create token (#3533) * [Feature] JVM parameter optimization , related issue #3370 * [Feature] JVM parameter optimization , related issue #3370 * test release 1.3.2 version rollback * test release 1.3.2 version rollback * test * test release 1.3.2 version rollback * modify tag 1.3.0 to HEAD * modify general user can't create token Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * if task is null , set task type is null instead of "null" * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE (#3537) * [Fix-3536][api]If user didn't have tenant,create resource will NPE * [Fix-3536][api]If user didn't have tenant,create resource directory will NPE * modify general user can't create,delete,update token (#3538) Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * [ui]Resource delete OK button to increase loading, change the number of homepage display cursor * fix * [Fix-3616][Server] when worker akc/response master exception , async retry (#3748) * [fixbug][ui]Repair the master and worker management instrument display * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode (#3741) * [fixbug][ui]Remove non-existent or deleted resources disabled * [fixBug-3621][ui]If the workflow instance status is executing status, it is forbidden to select * [fix-3553][ui]Repair click workflow connection, select the entire path * fix * fix * [Fix-3238][docker]Fix that can not create folder in docker with standalone mode * [Fix-3616][Server] when worker akc/response master exception , async retry (#3776) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Improvement-3720][ui] js mailbox verification fix * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3784) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3124][docker]Fix that can not build a docker image on windows (#3765) * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect (#3786) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> * [Fix-3258][Security][Worker group manage] Connot get create time and update time,report DateTimeParseException (#3787) * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * [Fix-3616][Server] when worker akc/response master exception , async retry * The batch delete function in the workflow definition and workflow instance pages cannot be canceled if selected. * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [Fix-3549] [Server][sqlTask]The alias column in the query SQL does not take effect * [BugFixed] issue #3258 (#3265) * 'ExecutionStatus' * '3258' * Update WorkerGroupServiceTest.java * Delete UserState.java * Delete ResourceSyncService.java * Delete core-site.xml * Delete hdfs-site.xml Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> * [fixBug-3792][ui]Click on the sidebar to adapt the width of the pie chart on the project homepage * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed * [Fix-#3713][common]Fix that catfile method Stream not closed (#3810) * [Bug-3713][HadoopUtils] catfile method Stream not closed (#3715) * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * fix bug Delete invalid field: executorcores Modify verification prompt * dag add close button * reset last version * reset last version * dag add close buttion dag add close buttion * update CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * updae CLICK_SAVE_WORKFLOW_BUTTON xpath * Update CreateWorkflowLocator.java modify submit workflow button * Update CreateWorkflowLocator.java * Update CreateWorkflowLocator.java modify CLICK_ADD_BUTTON * Update CreateWorkflowLocator.java delete print * Update CreateWorkflowLocator.java 1 * Update CreateWorkflowLocator.java 1 * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dag.vue * Update dag.vue * Update dag.vue * Update CreateWorkflowLocator.java * Revert "Setting '-XX:+DisableExplicitGC ' causes netty memory leaks" This reverts commit 3a2cba7a * Setting '-XX:+DisableExplicitGC ' causes netty memory leaks in addition update '- XX: largepagesizeinbytes = 128M' to '- XX: largepagesizeinbytes = 10M' * Update dolphinscheduler-daemon.sh * catfile method Stream not closed * catfile method Stream not closed Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3713][common]Fix that catfile method Stream not closed Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> * [Fix-#3487][api、dao] cherry pick from dev to fix that create folder duplicate name under multithreading * [Hotfix-3131][api] Fix the new tenant already exists prompt (#3132) * Bugfix: Fix the new tenant already exists prompt * Feature: Add test cases * Update TenantServiceTest.java Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> * Set up JDK 11 for SonarCloud in github action. (#3052) * Set up JDK 11 for SonarCloud in github action. * Fix javadoc error with JDK 11. * Prevent Javadoc from stopping if it finds any html errors. * [fixBug-3621][ui]Select the batch checkbox to unfilter the instances in the executing state * add verify tenant name cannot contain special characters. * [fixBug-3840][ui]The tenant code only allows letters or a combination of letters and numbers * fix * fix * fix * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it need replace the origin resource file. (#3862) * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [Fix-#3702][api] When re-upload the resource file but don't change the name or desc,it will not replace the origin resource file. * [fixbug-3621][ui]Workflow instance ready to stop and ready to suspend state prohibits checking * [fixbug-3887][ui]Fix missing English translation of re-upload files * add process define name verify. (#3879) * Revert "[1.3.3-release][fix-3835][ui] When the tenantName contains "<", the tenant drop-down list is blankadd verify tenant name cannot contain special characters." * revert pr 3872 * [FIX-3617][Service]after subtask fault tolerance, 2 task instances are generated (#3830) * fix bug(#3617): after subtask fault tolerance, 2 task instances are generated. * delete unused code * update code smell * refactor sub work command process * add process service ut * add license header * fix some code smell * chang ut java8 to java11 * update sonar to java11 * copy ut config from dev * remove checkstyle * revert to 1.3.3 * change proess service test to executor service * add process service test * add process service test * revert * revert * add comments * change dev to 1.3.3-release * revert Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3487][sql] add dolphinscheduler_dml.sql under 1.3.3_schema (#3907) * [FIX-3836][1.3.3-release-API] process definition validation name interface prompt information error (#3899) * fix bug : error message * fix code smell * fix code smell * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat (#3913) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat Co-authored-by: Kirs <acm_master@163.com> * Repair check box cannot be canceled * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly (#3918) * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [FIX_#3789][remote]cherry pick from dev to support netty heart beat * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly * [fix-3843][api] When update workflow definition,if name already exists, the prompt is not friendly Co-authored-by: Kirs <acm_master@163.com> * [Fix-#3487][sql] update uc_dolphin_T_t_ds_resources_un * Workflow definition name re-modified and added check * [fix-#3843][api]When update workflow definition,if name already exists, the prompt is not friendly. * update code. * [#3931][ui]Field name optimization for spark, flink, and mr * change version from 1.3.2-SNAPSHOT to 1.3.3-SNAPSHOT (#3934) * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [ambari-plugin]change version 1.3.2 to 1.3.3 (#3935) * fix bug:3615 After the task is executed successfully, but the next task has not been submitted, stop the master * [fixBug-3964][ui]Switch back and forth over timeout alarm, the selected value is empty * solve too many files, close logClientService (#3971) * fix #3966 sub process doesnot send alert mail after process instance ending. (#3972) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [Fix-#3618][server] resolve task executed finished but not release the file handle (#3975) * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3618][server] resolve task executed finished but not release the file handle * [Fix-#3958][api] files should not be created successfully in the directory of the authorized file * [FIX-3966] The timeout warning does not take effect in sub_process (#3982) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent Co-authored-by: baoliang <baoliang@analysys.com.cn> * fix import dolphinscheduler_mysql.sql insert admin user data * [FIX-3929] condition task would post wrong tasks when failover. (#3999) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge Co-authored-by: baoliang <baoliang@analysys.com.cn> * [FIX-3929] because of no lock, start up failover would dispatch two same tasks. (#4004) * fix #3966 sub process doesnot send alert mail after process instance ending. * fix bug 3964: sub_process The timeout warning does not take effect add timeout warning for sub_process/dependent task. * fix code smell * fix code smell * fix code smell * update worker group inherit from parent * remove stdout in logback configuration * fix bug #3929 condition task would post error when failover. * remove unused test * add comments * add skip node judge * fix bug 3929: because of no lock, start up failover would dispatch two same tasks. Co-authored-by: baoliang <baoliang@analysys.com.cn> * revert pom version to 1.3.3-release * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * [release]revert pom version to 1.3.3-release * fix bug 4010: remove failed condition tasks from error-task-list. (#4011) Co-authored-by: baoliang <baoliang@analysys.com.cn> * [maven-release-plugin] prepare release 1.3.3 * [maven-release-plugin] prepare for next development iteration * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut test * refactor ut test * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor ut * merge from 1.3.3-release * refactor ut * refactor ut * refactor * refactor * refactor code style * refactor code style * refactor code style * refactor code style * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * merge from 1.3.3-release * refactor code style Co-authored-by: break60 <790061044@qq.com> Co-authored-by: wuchunfu <319355703@qq.com> Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Co-authored-by: lenboo <baoliang@analysys.com.cn> Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn> Co-authored-by: Yelli <amarantine@my.com> Co-authored-by: Eights-Li <yelli.hl@gmail.com> Co-authored-by: JinyLeeChina <42576980+JinyLeeChina@users.noreply.github.com> Co-authored-by: dailidong <dailidong66@gmail.com> Co-authored-by: qiaozhanwei <qiaozhanwei@outlook.com> Co-authored-by: XiaotaoYi <v-xiayi@hotmail.com> Co-authored-by: Yichao Yang <1048262223@qq.com> Co-authored-by: zhuangchong <zhuangchong8@163.com> Co-authored-by: BoYiZhang <39816903+BoYiZhang@users.noreply.github.com> Co-authored-by: muzhongjiang <mu_zhongjiang@163.com> Co-authored-by: Jave-Chen <baicai.chen@gmail.com> Co-authored-by: zhuangchong <zhuangchong6@163.com> Co-authored-by: zhuangchong <37063904+zhuangchong@users.noreply.github.com> Co-authored-by: Kirs <acm_master@163.com> Co-authored-by: lgcareer <lgcareer@apache.org> Co-authored-by: wulingqi <wulingqi@baijiahulian.com>
4 years ago
){
boolean result = false;
Set<String> subsequentNodes = dag.getSubsequentNodes(parentNodeName);
if(CollectionUtils.isEmpty(subsequentNodes)){
return result;
}
for(String nodeName : subsequentNodes){
TaskNode taskNode = dag.getNode(nodeName);
List<String> preTasksList = JSONUtils.toList(taskNode.getPreTasks(), String.class);
if(preTasksList.contains(parentNodeName) && taskNode.isConditionsTask()){
return true;
}
}
return result;
}
/**
* is there have conditions after the parent node
* @param parentNodeName
* @return
*/
public static boolean haveConditionsAfterNode(String parentNodeName,
List<TaskNode> taskNodes
){
boolean result = false;
if(CollectionUtils.isEmpty(taskNodes)){
return result;
}
for(TaskNode taskNode : taskNodes){
List<String> preTasksList = JSONUtils.toList(taskNode.getPreTasks(), String.class);
if(preTasksList.contains(parentNodeName) && taskNode.isConditionsTask()){
return true;
}
}
return result;
}
}