You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by zh...@apache.org on 2022/05/16 12:18:18 UTC

[dolphinscheduler] branch 3.0.0-beta-prepare updated (b6b1fffbd6 -> 1b4036e2ce)

This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a change to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git


    from b6b1fffbd6 [docker] Change docker hub HUB
     new 104f67d306 [Bug] [MASTER-9811]fix cmd param to overwrite global param when executing complement (#9952)
     new 010d59f1d6 [Fix-10002] Fix some bugs in datasource list (#10005)
     new 0da0613a1b [doc] Add the description about execute type in SQL task (#9987)
     new ca0f41c10f [Fix][UI] Fix the task name validator error. (#10008)
     new 7b0e6fe5ec [Fix-9975] The selected task instance was recreated when the Master service fail… (#9976)
     new 2eb8c626e3 [Fix][UI] Support only one file upload on the file manage page. (#10007)
     new 665e62af44 [doc] Add example and notice about task type Dependent (#10001)
     new 931770be70 [Fix][UI] Fix errorOutputPath column in dataquality page (#10015)
     new 5e1229f007 [doc] Correct kubernetes (#9985)
     new 44c6a053e7 [fix-9991][worker]fix statement is closed before resultSet.getMetaData() (#10014)
     new 25538aaf67 [dataquality] Fix task commnd null bug (#9974)
     new ea284526f9 [doc] Correct docs of development-environment-setup (#9995)
     new d74aa47196 [Fix-10039] Flink run command when perfecting Python jobs (#10042)
     new 0f841d6ff9 [Bug] fix run on docker and k8s (#10026)
     new 1b4036e2ce [python] Fix pre_submit_check error when local params in tasks (#10032)

The 15 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 deploy/docker/docker-compose.yml                   |  22 ++++-----
 .../deployment-dolphinscheduler-alert.yaml         |   2 +
 .../templates/deployment-dolphinscheduler-api.yaml |   6 ++-
 .../job-dolphinscheduler-schema-initializer.yaml   |   4 +-
 .../statefulset-dolphinscheduler-master.yaml       |   2 +
 .../statefulset-dolphinscheduler-worker.yaml       |   2 +
 .../development/development-environment-setup.md   |  43 +++++++-----------
 docs/docs/en/guide/installation/kubernetes.md      |   2 +-
 docs/docs/en/guide/start/docker.md                 |  17 +++++--
 docs/docs/en/guide/task/dependent.md               |  48 ++++++++++++--------
 docs/docs/en/guide/task/sql.md                     |   7 ++-
 .../development/development-environment-setup.md   |  49 +++++++++------------
 docs/docs/zh/guide/installation/kubernetes.md      |   2 +-
 docs/docs/zh/guide/start/docker.md                 |  17 +++++--
 docs/docs/zh/guide/task/dependent.md               |  49 +++++++++++++--------
 docs/docs/zh/guide/task/sql.md                     |   7 ++-
 docs/img/depend-node-en.png                        | Bin 496668 -> 0 bytes
 docs/img/depend-node.png                           | Bin 123401 -> 0 bytes
 docs/img/depend-node1-en.png                       | Bin 516594 -> 0 bytes
 docs/img/depend-node2.png                          | Bin 120491 -> 0 bytes
 docs/img/depend-node3-en.png                       | Bin 479533 -> 0 bytes
 docs/img/depend-node3.png                          | Bin 140998 -> 0 bytes
 docs/img/dependent-nodes-en.png                    | Bin 95552 -> 0 bytes
 docs/img/dependent_edit.png                        | Bin 18061 -> 0 bytes
 docs/img/tasks/demo/dependent_task01.png           | Bin 0 -> 159694 bytes
 docs/img/tasks/demo/dependent_task02.png           | Bin 0 -> 159538 bytes
 docs/img/tasks/demo/dependent_task03.png           | Bin 0 -> 164972 bytes
 .../img/tasks/icons}/dependent.png                 | Bin
 docs/img/tasks/icons/python.png                    | Bin 34066 -> 1618 bytes
 .../master/runner/WorkflowExecuteThread.java       |  41 +++++++++++++++++
 .../pydolphinscheduler/core/process_definition.py  |   6 ++-
 .../src/pydolphinscheduler/tasks/switch.py         |   6 ++-
 .../tests/core/test_process_definition.py          |  29 +++++++++++-
 .../service/process/ProcessServiceImpl.java        |   5 ++-
 .../plugin/task/dq/DataQualityTask.java            |  17 ++++---
 .../plugin/task/flink/FlinkConstants.java          |   1 +
 .../plugin/task/flink/FlinkTask.java               |   4 ++
 .../dolphinscheduler/plugin/task/sql/SqlTask.java  |  22 +++------
 .../src/main/bin/upgrade-schema.sh                 |   4 +-
 dolphinscheduler-tools/src/main/docker/Dockerfile  |   3 +-
 .../views/data-quality/task-result/use-table.ts    |   2 +-
 .../src/views/datasource/list/json-highlight.tsx   |  10 +++--
 .../src/views/datasource/list/use-columns.ts       |   2 +-
 .../task/components/node/fields/use-dependent.ts   |  12 ++---
 .../views/projects/task/components/node/types.ts   |   4 +-
 .../src/views/resource/file/upload/index.tsx       |  16 ++++++-
 .../src/views/resource/file/upload/use-form.ts     |   1 +
 47 files changed, 305 insertions(+), 159 deletions(-)
 delete mode 100644 docs/img/depend-node-en.png
 delete mode 100644 docs/img/depend-node.png
 delete mode 100644 docs/img/depend-node1-en.png
 delete mode 100644 docs/img/depend-node2.png
 delete mode 100644 docs/img/depend-node3-en.png
 delete mode 100644 docs/img/depend-node3.png
 delete mode 100644 docs/img/dependent-nodes-en.png
 delete mode 100644 docs/img/dependent_edit.png
 create mode 100644 docs/img/tasks/demo/dependent_task01.png
 create mode 100644 docs/img/tasks/demo/dependent_task02.png
 create mode 100644 docs/img/tasks/demo/dependent_task03.png
 copy {dolphinscheduler-ui/public/images/task-icons => docs/img/tasks/icons}/dependent.png (100%)


[dolphinscheduler] 13/15: [Fix-10039] Flink run command when perfecting Python jobs (#10042)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit d74aa47196e8016895619977a2ae52a0b8d3007a
Author: Dannila <94...@users.noreply.github.com>
AuthorDate: Mon May 16 15:06:21 2022 +0800

    [Fix-10039] Flink run command when perfecting Python jobs (#10042)
    
    * [fix] flink task
    
    * [fix] flink task
    
    (cherry picked from commit d643e1c1cf27d6acfc46deb655e0b5f2f813cb24)
---
 .../org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java | 1 +
 .../java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java | 4 ++++
 2 files changed, 5 insertions(+)

diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
index 2e55de9b25..42cb5ad78c 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
@@ -50,6 +50,7 @@ public class FlinkConstants {
     public static final String FLINK_MAIN_CLASS = "-c";
     public static final String FLINK_PARALLELISM = "-p";
     public static final String FLINK_SHUTDOWN_ON_ATTACHED_EXIT = "-sae";
+    public static final String FLINK_PYTHON = "-py";
 
     public static final String FLINK_FORMAT_EXECUTION_TARGET = "set execution.target=%s";
     public static final String FLINK_FORMAT_YARN_APPLICATION_NAME = "set yarn.application.name=%s";
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
index 966e8a01bd..f70a3d67b7 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
@@ -149,6 +149,10 @@ public class FlinkTask extends AbstractYarnTask {
 
         ResourceInfo mainJar = flinkParameters.getMainJar();
         if (mainJar != null) {
+            // -py
+            if(ProgramType.PYTHON == programType) {
+                args.add(FlinkConstants.FLINK_PYTHON);
+            }
             args.add(mainJar.getRes());
         }
 


[dolphinscheduler] 08/15: [Fix][UI] Fix errorOutputPath column in dataquality page (#10015)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 931770be702ef5721449211fac87dfea2303551a
Author: rockfang <65...@qq.com>
AuthorDate: Fri May 13 10:16:54 2022 +0800

    [Fix][UI] Fix errorOutputPath column in dataquality page (#10015)
    
    
    (cherry picked from commit d17379da1f822d0ccc01f737c3037c18416238db)
---
 dolphinscheduler-ui/src/views/data-quality/task-result/use-table.ts | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dolphinscheduler-ui/src/views/data-quality/task-result/use-table.ts b/dolphinscheduler-ui/src/views/data-quality/task-result/use-table.ts
index cfcb380836..e5472e842f 100644
--- a/dolphinscheduler-ui/src/views/data-quality/task-result/use-table.ts
+++ b/dolphinscheduler-ui/src/views/data-quality/task-result/use-table.ts
@@ -166,7 +166,7 @@ export function useTable() {
         title: t('data_quality.task_result.error_output_path'),
         key: 'errorOutputPath',
         render: (row: ResultItem) => {
-          return row.errorOutputPath ? row : '-'
+          return row.errorOutputPath ? row.errorOutputPath : '-'
         },
         width: 200
       },


[dolphinscheduler] 15/15: [python] Fix pre_submit_check error when local params in tasks (#10032)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 1b4036e2ce55ae62575a6dc601cccecf3c1f832c
Author: 陈家名 <13...@163.com>
AuthorDate: Mon May 16 19:51:41 2022 +0800

    [python] Fix pre_submit_check error when local params in tasks (#10032)
    
    
    (cherry picked from commit a2cd2a99e65c3a85bdebfc37e1465e98b2d4431a)
---
 .../pydolphinscheduler/core/process_definition.py  |  6 +++--
 .../src/pydolphinscheduler/tasks/switch.py         |  6 ++++-
 .../tests/core/test_process_definition.py          | 29 +++++++++++++++++++++-
 3 files changed, 37 insertions(+), 4 deletions(-)

diff --git a/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/process_definition.py b/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/process_definition.py
index 649e2ce51a..dbf2c41795 100644
--- a/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/process_definition.py
+++ b/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/process_definition.py
@@ -350,14 +350,16 @@ class ProcessDefinition(Base):
 
         This method should be called before process definition submit to java gateway
         For now, we have below checker:
-        * `self.param` should be set if task `switch` in this workflow.
+        * `self.param` or at least one local param of task should be set if task `switch` in this workflow.
         """
         if (
             any([task.task_type == TaskType.SWITCH for task in self.tasks.values()])
             and self.param is None
+            and all([len(task.local_params) == 0 for task in self.tasks.values()])
         ):
             raise PyDSParamException(
-                "Parameter param must be provider if task Switch in process definition."
+                "Parameter param or at least one local_param of task must "
+                "be provider if task Switch in process definition."
             )
 
     def submit(self) -> int:
diff --git a/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/tasks/switch.py b/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/tasks/switch.py
index 28032f88e7..0c9a2b82b3 100644
--- a/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/tasks/switch.py
+++ b/dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/tasks/switch.py
@@ -129,7 +129,11 @@ class SwitchCondition(Base):
 
 
 class Switch(Task):
-    """Task switch object, declare behavior for switch task to dolphinscheduler."""
+    """Task switch object, declare behavior for switch task to dolphinscheduler.
+
+    Param of process definition or at least one local param of task must be set
+    if task `switch` in this workflow.
+    """
 
     def __init__(self, name: str, condition: SwitchCondition, *args, **kwargs):
         super().__init__(name, TaskType.SWITCH, *args, **kwargs)
diff --git a/dolphinscheduler-python/pydolphinscheduler/tests/core/test_process_definition.py b/dolphinscheduler-python/pydolphinscheduler/tests/core/test_process_definition.py
index 88028f72c7..36e1cb035e 100644
--- a/dolphinscheduler-python/pydolphinscheduler/tests/core/test_process_definition.py
+++ b/dolphinscheduler-python/pydolphinscheduler/tests/core/test_process_definition.py
@@ -240,11 +240,38 @@ def test__pre_submit_check_switch_without_param(mock_code_version):
         parent >> switch
         with pytest.raises(
             PyDSParamException,
-            match="Parameter param must be provider if task Switch in process definition.",
+            match="Parameter param or at least one local_param of task must "
+            "be provider if task Switch in process definition.",
         ):
             pd._pre_submit_check()
 
 
+@patch(
+    "pydolphinscheduler.core.task.Task.gen_code_and_version",
+    return_value=(123, 1),
+)
+def test__pre_submit_check_switch_with_local_params(mock_code_version):
+    """Test :func:`_pre_submit_check` if process definition with switch with local params of task."""
+    with ProcessDefinition(TEST_PROCESS_DEFINITION_NAME) as pd:
+        parent = Task(
+            name="parent",
+            task_type=TEST_TASK_TYPE,
+            local_params=[
+                {"prop": "var", "direct": "OUT", "type": "VARCHAR", "value": ""}
+            ],
+        )
+        switch_child_1 = Task(name="switch_child_1", task_type=TEST_TASK_TYPE)
+        switch_child_2 = Task(name="switch_child_2", task_type=TEST_TASK_TYPE)
+        switch_condition = SwitchCondition(
+            Branch(condition="${var} > 1", task=switch_child_1),
+            Default(task=switch_child_2),
+        )
+
+        switch = Switch(name="switch", condition=switch_condition)
+        parent >> switch
+        pd._pre_submit_check()
+
+
 def test_process_definition_get_define_without_task():
     """Test process definition function get_define without task."""
     expect = {


[dolphinscheduler] 11/15: [dataquality] Fix task commnd null bug (#9974)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 25538aaf67d1db27700f1cc6948de47c1722566c
Author: zixi0825 <su...@gmail.com>
AuthorDate: Sun May 15 17:46:31 2022 +0800

    [dataquality] Fix task commnd null bug (#9974)
    
    
    (cherry picked from commit 359cbe2a1ea247a8a567762100eb80be51679c75)
---
 .../plugin/task/dq/DataQualityTask.java                 | 17 +++++++++++------
 1 file changed, 11 insertions(+), 6 deletions(-)

diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-dataquality/src/main/java/org/apache/dolphinscheduler/plugin/task/dq/DataQualityTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-dataquality/src/main/java/org/apache/dolphinscheduler/plugin/task/dq/DataQualityTask.java
index d0575573a1..039987da0e 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-dataquality/src/main/java/org/apache/dolphinscheduler/plugin/task/dq/DataQualityTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-dataquality/src/main/java/org/apache/dolphinscheduler/plugin/task/dq/DataQualityTask.java
@@ -42,6 +42,7 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters
 import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
 import org.apache.dolphinscheduler.plugin.task.api.parser.ParameterUtils;
 import org.apache.dolphinscheduler.plugin.task.api.utils.ArgsUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
 import org.apache.dolphinscheduler.plugin.task.dq.rule.RuleManager;
 import org.apache.dolphinscheduler.plugin.task.dq.rule.parameter.DataQualityConfiguration;
 import org.apache.dolphinscheduler.plugin.task.dq.utils.spark.SparkArgsUtils;
@@ -52,6 +53,7 @@ import java.io.File;
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -160,8 +162,6 @@ public class DataQualityTask extends AbstractYarnTask {
         List<String> args = new ArrayList<>();
 
         args.add(SPARK2_COMMAND);
-
-        // other parameters
         args.addAll(SparkArgsUtils.buildArgs(dataQualityParameters.getSparkParameters()));
 
         // replace placeholder
@@ -169,10 +169,15 @@ public class DataQualityTask extends AbstractYarnTask {
 
         String command = null;
 
-        if (null != paramsMap) {
-            command = ParameterUtils.convertParameterPlaceholders(String.join(" ", args), ParamUtils.convert(paramsMap));
+        if (MapUtils.isEmpty(paramsMap)) {
+            paramsMap = new HashMap<>();
+        }
+
+        if (MapUtils.isNotEmpty(dqTaskExecutionContext.getParamsMap())) {
+            paramsMap.putAll(dqTaskExecutionContext.getParamsMap());
         }
 
+        command = ParameterUtils.convertParameterPlaceholders(String.join(" ", args), ParamUtils.convert(paramsMap));
         logger.info("data quality task command: {}", command);
 
         return command;
@@ -181,8 +186,8 @@ public class DataQualityTask extends AbstractYarnTask {
     @Override
     protected void setMainJarName() {
         ResourceInfo mainJar = new ResourceInfo();
-        String basePath = System.getProperty("user.dir").replace(File.separator + "bin", File.separator + "libs");
-        mainJar.setRes(basePath + File.separator + CommonUtils.getDataQualityJarName());
+        String basePath = System.getProperty("user.dir").replace(File.separator + "bin", "");
+        mainJar.setRes(basePath + File.separator + "libs" + File.separator + CommonUtils.getDataQualityJarName());
         dataQualityParameters.getSparkParameters().setMainJar(mainJar);
     }
 


[dolphinscheduler] 01/15: [Bug] [MASTER-9811]fix cmd param to overwrite global param when executing complement (#9952)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 104f67d30689007278a6bb9f2e050521798258e6
Author: Tq <ti...@gmail.com>
AuthorDate: Wed May 11 18:37:03 2022 +0800

    [Bug] [MASTER-9811]fix cmd param to overwrite global param when executing complement (#9952)
    
    * fix cmd param to overwrite global param when executing complement
    
    * fix cmd param to overwrite global param when executing complement
    
    (cherry picked from commit d4aeee16e5f3f67e37a31fa606acf0b313189655)
---
 .../master/runner/WorkflowExecuteThread.java       | 38 ++++++++++++++++++++++
 .../service/process/ProcessServiceImpl.java        |  5 +--
 2 files changed, 41 insertions(+), 2 deletions(-)

diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
index e4f09e5a9c..98556dc0f0 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
@@ -24,6 +24,8 @@ import static org.apache.dolphinscheduler.common.Constants.CMD_PARAM_RECOVER_PRO
 import static org.apache.dolphinscheduler.common.Constants.CMD_PARAM_START_NODES;
 import static org.apache.dolphinscheduler.common.Constants.DEFAULT_WORKER_GROUP;
 import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_BLOCKING;
+import static org.apache.dolphinscheduler.plugin.task.api.enums.DataType.VARCHAR;
+import static org.apache.dolphinscheduler.plugin.task.api.enums.Direct.IN;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
@@ -942,6 +944,9 @@ public class WorkflowExecuteThread {
         if (processInstance.isComplementData() && complementListDate.size() == 0) {
             Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
             if (cmdParam != null && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)) {
+                // reset global params while there are start parameters
+                setGlobalParamIfCommanded(processDefinition, cmdParam);
+
                 Date start = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
                 Date end = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
                 List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(processInstance.getProcessDefinitionCode());
@@ -1960,4 +1965,37 @@ public class WorkflowExecuteThread {
         }
     }
 
+    private void setGlobalParamIfCommanded(ProcessDefinition processDefinition, Map<String, String> cmdParam) {
+        // get start params from command param
+        Map<String, String> startParamMap = new HashMap<>();
+        if (cmdParam.containsKey(Constants.CMD_PARAM_START_PARAMS)) {
+            String startParamJson = cmdParam.get(Constants.CMD_PARAM_START_PARAMS);
+            startParamMap = JSONUtils.toMap(startParamJson);
+        }
+        Map<String, String> fatherParamMap = new HashMap<>();
+        if (cmdParam.containsKey(Constants.CMD_PARAM_FATHER_PARAMS)) {
+            String fatherParamJson = cmdParam.get(Constants.CMD_PARAM_FATHER_PARAMS);
+            fatherParamMap = JSONUtils.toMap(fatherParamJson);
+        }
+        startParamMap.putAll(fatherParamMap);
+        // set start param into global params
+        Map<String, String> globalMap = processDefinition.getGlobalParamMap();
+        List<Property> globalParamList = processDefinition.getGlobalParamList();
+        if (startParamMap.size() > 0 && globalMap != null) {
+            //start param to overwrite global param
+            for (Map.Entry<String, String> param : globalMap.entrySet()) {
+                String val = startParamMap.get(param.getKey());
+                if (val != null) {
+                    param.setValue(val);
+                }
+            }
+            //start param to create new global param if global not exist
+            for (Map.Entry<String, String> startParam : startParamMap.entrySet()) {
+                if (!globalMap.containsKey(startParam.getKey())) {
+                    globalMap.put(startParam.getKey(), startParam.getValue());
+                    globalParamList.add(new Property(startParam.getKey(), IN, VARCHAR, startParam.getValue()));
+                }
+            }
+        }
+    }
 }
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
index 53c20a5ae8..5136891280 100644
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
@@ -818,14 +818,15 @@ public class ProcessServiceImpl implements ProcessService {
         // set start param into global params
         Map<String, String> globalMap = processDefinition.getGlobalParamMap();
         List<Property> globalParamList = processDefinition.getGlobalParamList();
-        if (startParamMap.size() > 0
-            && globalMap != null) {
+        if (startParamMap.size() > 0 && globalMap != null) {
+            //start param to overwrite global param
             for (Map.Entry<String, String> param : globalMap.entrySet()) {
                 String val = startParamMap.get(param.getKey());
                 if (val != null) {
                     param.setValue(val);
                 }
             }
+            //start param to create new global param if global not exist
             for (Entry<String, String> startParam : startParamMap.entrySet()) {
                 if (!globalMap.containsKey(startParam.getKey())) {
                     globalMap.put(startParam.getKey(), startParam.getValue());


[dolphinscheduler] 03/15: [doc] Add the description about execute type in SQL task (#9987)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 0da0613a1bbea99b05103c788b913dcb1defb34a
Author: Kerwin <37...@users.noreply.github.com>
AuthorDate: Thu May 12 14:42:39 2022 +0800

    [doc] Add the description about execute type in SQL task (#9987)
    
    
    (cherry picked from commit bce5a284409e13d477c2f21b40a0db7c895e63f0)
---
 docs/docs/en/guide/task/sql.md | 7 ++++++-
 docs/docs/zh/guide/task/sql.md | 7 ++++++-
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/docs/docs/en/guide/task/sql.md b/docs/docs/en/guide/task/sql.md
index 9c62245169..f40893ff22 100644
--- a/docs/docs/en/guide/task/sql.md
+++ b/docs/docs/en/guide/task/sql.md
@@ -16,7 +16,12 @@ Refer to [DataSource](../datasource/introduction.md)
 ## Task Parameter
 
 - Data source: Select the corresponding DataSource.
-- SQL type: Supports query and non-query. The query is a `select` type query, which is returned with a result set. You can specify three templates for email notification: form, attachment or form attachment. Non-queries return without a result set, three types of operations are: update, delete and insert.
+- SQL type: Supports query and non-query. 
+  - Query: supports `DML select` type commands, which return a result set. You can specify three templates for email notification as form, attachment or form attachment;
+  - Non-query: support `DDL` all commands and `DML update, delete, insert` three types of commands;
+    - Segmented execution symbol: When the data source does not support executing multiple SQL statements at a time, the symbol for splitting SQL statements is provided to call the data source execution method multiple times.
+    Example: 1. When the Hive data source is selected as the data source, this parameter does not need to be filled in. Because the Hive data source itself supports executing multiple SQL statements at one time;
+             2. When the MySQL data source is selected as the data source, and multi-segment SQL statements are to be executed, this parameter needs to be filled in with a semicolon `;`. Because the MySQL data source does not support executing multiple SQL statements at one time;
 - SQL parameter: The input parameter format is `key1=value1;key2=value2...`.
 - SQL statement: SQL statement.
 - UDF function: For Hive DataSources, you can refer to UDF functions created in the resource center, but other DataSource do not support UDF functions.
diff --git a/docs/docs/zh/guide/task/sql.md b/docs/docs/zh/guide/task/sql.md
index 0c1c673356..4896e80bf9 100644
--- a/docs/docs/zh/guide/task/sql.md
+++ b/docs/docs/zh/guide/task/sql.md
@@ -16,7 +16,12 @@ SQL任务类型,用于连接数据库并执行相应SQL。
 ## 任务参数
 
 - 数据源:选择对应的数据源
-- sql类型:支持查询和非查询两种,查询是select类型的查询,是有结果集返回的,可以指定邮件通知为表格、附件或表格附件三种模板。非查询是没有结果集返回的,是针对update、delete、insert三种类型的操作。
+- sql类型:支持查询和非查询两种。
+  - 查询:支持 `DML select` 类型的命令,是有结果集返回的,可以指定邮件通知为表格、附件或表格附件三种模板;
+  - 非查询:支持 `DDL`全部命令 和 `DML update、delete、insert` 三种类型的命令;
+    - 分段执行符号:提供在数据源不支持一次执行多段SQL语句时,拆分SQL语句的符号来进行多次调用数据源执行方法。
+    例子:1.当数据源选择Hive数据源时,不需要填写此参数。因为Hive数据源本身支持一次执行多段SQL语句;
+         2.当数据源选择MySQL数据源时,并且要执行多段SQL语句时,需要填写此参数为分号 `;`。因为MySQL数据源不支持一次执行多段SQL语句;
 - sql参数:输入参数格式为key1=value1;key2=value2…
 - sql语句:SQL语句
 - UDF函数:对于HIVE类型的数据源,可以引用资源中心中创建的UDF函数,其他类型的数据源暂不支持UDF函数。


[dolphinscheduler] 09/15: [doc] Correct kubernetes (#9985)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 5e1229f007c6d9c00660e9a13af34e8b349ec806
Author: 旺阳 <wa...@lqwang.net>
AuthorDate: Fri May 13 10:39:05 2022 +0800

    [doc] Correct kubernetes (#9985)
    
    Co-authored-by: qingwli <qi...@cisco.com>
    (cherry picked from commit ee2c516813a6cd16c181d2101878983494d0b2bc)
---
 docs/docs/en/guide/installation/kubernetes.md | 2 +-
 docs/docs/zh/guide/installation/kubernetes.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/docs/en/guide/installation/kubernetes.md b/docs/docs/en/guide/installation/kubernetes.md
index f56da053a3..572bf19e7b 100644
--- a/docs/docs/en/guide/installation/kubernetes.md
+++ b/docs/docs/en/guide/installation/kubernetes.md
@@ -18,7 +18,7 @@ To publish the release name `dolphinscheduler` version, please execute the follo
 
 ```
 $ tar -zxvf apache-dolphinscheduler-3.0.0-beta-1-src.tar.gz
-$ cd apache-dolphinscheduler-3.0.0-beta-1-src/docker/kubernetes/dolphinscheduler
+$ cd apache-dolphinscheduler-3.0.0-beta-1-src/deploy/kubernetes/dolphinscheduler
 $ helm repo add bitnami https://charts.bitnami.com/bitnami
 $ helm dependency update .
 $ helm install dolphinscheduler . --set image.tag=3.0.0-beta-1
diff --git a/docs/docs/zh/guide/installation/kubernetes.md b/docs/docs/zh/guide/installation/kubernetes.md
index c5c5855752..5a97ed2467 100644
--- a/docs/docs/zh/guide/installation/kubernetes.md
+++ b/docs/docs/zh/guide/installation/kubernetes.md
@@ -19,7 +19,7 @@ Kubernetes部署目的是在Kubernetes集群中部署 DolphinScheduler 服务,
 
 ```
 $ tar -zxvf apache-dolphinscheduler-3.0.0-beta-1-src.tar.gz
-$ cd apache-dolphinscheduler-3.0.0-beta-1-src/docker/kubernetes/dolphinscheduler
+$ cd apache-dolphinscheduler-3.0.0-beta-1-src/deploy/kubernetes/dolphinscheduler
 $ helm repo add bitnami https://charts.bitnami.com/bitnami
 $ helm dependency update .
 $ helm install dolphinscheduler . --set image.tag=3.0.0-beta-1


[dolphinscheduler] 07/15: [doc] Add example and notice about task type Dependent (#10001)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 665e62af44b9457104cc871cbdfa35beeb3fd28f
Author: QuakeWang <45...@users.noreply.github.com>
AuthorDate: Thu May 12 18:23:43 2022 +0800

    [doc] Add example and notice about task type Dependent (#10001)
    
    Co-authored-by: Jiajie Zhong <zh...@gmail.com>
    (cherry picked from commit 0fe75482a21c56980f1c7f503bb75eaefb6417b3)
---
 docs/docs/en/guide/task/dependent.md     |  48 ++++++++++++++++++------------
 docs/docs/zh/guide/task/dependent.md     |  49 +++++++++++++++++++------------
 docs/img/depend-node-en.png              | Bin 496668 -> 0 bytes
 docs/img/depend-node.png                 | Bin 123401 -> 0 bytes
 docs/img/depend-node1-en.png             | Bin 516594 -> 0 bytes
 docs/img/depend-node2.png                | Bin 120491 -> 0 bytes
 docs/img/depend-node3-en.png             | Bin 479533 -> 0 bytes
 docs/img/depend-node3.png                | Bin 140998 -> 0 bytes
 docs/img/dependent-nodes-en.png          | Bin 95552 -> 0 bytes
 docs/img/dependent_edit.png              | Bin 18061 -> 0 bytes
 docs/img/tasks/demo/dependent_task01.png | Bin 0 -> 159694 bytes
 docs/img/tasks/demo/dependent_task02.png | Bin 0 -> 159538 bytes
 docs/img/tasks/demo/dependent_task03.png | Bin 0 -> 164972 bytes
 docs/img/tasks/icons/dependent.png       | Bin 0 -> 743 bytes
 docs/img/tasks/icons/python.png          | Bin 34066 -> 1618 bytes
 15 files changed, 61 insertions(+), 36 deletions(-)

diff --git a/docs/docs/en/guide/task/dependent.md b/docs/docs/en/guide/task/dependent.md
index ff86207451..1cbf354fd4 100644
--- a/docs/docs/en/guide/task/dependent.md
+++ b/docs/docs/en/guide/task/dependent.md
@@ -1,27 +1,39 @@
-# Dependent Node
+# Dependent
 
-- Dependent nodes are **dependency check nodes**. For example, process A depends on the successful execution of process B from yesterday, and the dependent node will check whether process B run successful yesterday.
+## Overview
 
-> Drag from the toolbar ![PNG](https://analysys.github.io/easyscheduler_docs_cn/images/toolbar_DEPENDENT.png) task node to the canvas, as shown in the figure below:
+Dependent nodes are **dependency check nodes**. For example, process A depends on the successful execution of process B from yesterday, and the dependent node will check whether process B run successful yesterday.
 
-<p align="center">
-   <img src="/img/dependent-nodes-en.png" width="80%" />
- </p>
 
-> The dependent node provides a logical judgment function, such as checking whether the B process was successful yesterday, or whether the C process was executed successfully.
+## Create Task
 
-  <p align="center">
-   <img src="/img/depend-node-en.png" width="80%" />
- </p>
+- Click `Project -> Management-Project -> Name-Workflow Definition`, and click the `Create Workflow` button to enter the DAG editing page.
+- Drag from the toolbar <img src="/img/tasks/icons/dependent.png" width="15"/> task node to canvas.
 
-> For example, process A is a weekly report task, processes B and C are daily tasks, and task A requires tasks B and C to be successfully executed every day of the last week, as shown in the figure:
+## Task Parameter
 
- <p align="center">
-   <img src="/img/depend-node1-en.png" width="80%" />
- </p>
+- **Node name**: The node name in a workflow definition is unique.
+- **Run flag**: Identifies whether this node schedules normally, if it does not need to execute, select the `prohibition execution`.
+- **Descriptive information**: Describe the function of the node.
+- **Task priority**: When the number of worker threads is insufficient, execute in the order of priority from high to low, and tasks with the same priority will execute in a first-in first-out order.
+- **Worker grouping**: Assign tasks to the machines of the worker group to execute. If `Default` is selected, randomly select a worker machine for execution.
+- **Environment Name**: Configure the environment name in which run the script.
+- **Times of failed retry attempts**: The number of times the task failed to resubmit.
+- **Failed retry interval**: The time interval (unit minute) for resubmitting the task after a failed task.
+- **Delayed execution time**: The time (unit minute) that a task delays in execution.
 
-> If the weekly report A also needs to be executed successfully last Tuesday:
+## Examples
 
- <p align="center">
-   <img src="/img/depend-node3-en.png" width="80%" />
- </p>
\ No newline at end of file
+The Dependent node provides a logical judgment function, which can detect the execution of the dependent node according to the logic.
+
+For example, process A is a weekly task, processes B and C are daily tasks, and task A requires tasks B and C to be successfully executed every day of the last week.
+
+![dependent_task01](/img/tasks/demo/dependent_task01.png)
+
+And another example is that process A is a weekly report task, processes B and C are daily tasks, and task A requires tasks B or C to be successfully executed every day of the last week:
+
+![dependent_task02](/img/tasks/demo/dependent_task02.png)
+
+If the weekly report A also needs to be executed successfully last Tuesday:
+
+![dependent_task03](/img/tasks/demo/dependent_task03.png)
diff --git a/docs/docs/zh/guide/task/dependent.md b/docs/docs/zh/guide/task/dependent.md
index abdc0435c6..b3fbd91f8d 100644
--- a/docs/docs/zh/guide/task/dependent.md
+++ b/docs/docs/zh/guide/task/dependent.md
@@ -1,27 +1,40 @@
-# 依赖节点
+# Dependent 节点
 
-- 依赖节点,就是**依赖检查节点**。比如A流程依赖昨天的B流程执行成功,依赖节点会去检查B流程在昨天是否有执行成功的实例。
+## 综述
 
-> 拖动工具栏中的![PNG](https://analysys.github.io/easyscheduler_docs_cn/images/toolbar_DEPENDENT.png)任务节点到画板中,如下图所示:
+Dependent 节点,就是**依赖检查节点**。比如 A 流程依赖昨天的 B 流程执行成功,依赖节点会去检查 B 流程在昨天是否有执行成功的实例。
 
-<p align="center">
-   <img src="/img/dependent_edit.png" width="80%" />
- </p>
+## 创建任务
 
-  > 依赖节点提供了逻辑判断功能,比如检查昨天的B流程是否成功,或者C流程是否执行成功。
+- 点击项目管理 -> 项目名称 -> 工作流定义,点击“创建工作流”按钮,进入 DAG 编辑页面;
+- 拖动工具栏的<img src="/img/tasks/icons/dependent.png" width="15"/> 任务节点到画板中。
 
-  <p align="center">
-   <img src="/img/depend-node.png" width="80%" />
- </p>
+## 任务参数
 
-  > 例如,A流程为周报任务,B、C流程为天任务,A任务需要B、C任务在上周的每一天都执行成功,如图示:
+- 节点名称:设置任务节点的名称。一个工作流定义中的节点名称是唯一的。
+- 运行标志:标识这个结点是否能正常调度,如果不需要执行,可以打开禁止执行开关。
+- 描述:描述该节点的功能。
+- 任务优先级:worker 线程数不足时,根据优先级从高到低依次执行,优先级一样时根据先进先出原则执行。
+- Worker 分组:任务分配给 worker 组的机器执行,选择 Default ,会随机选择一台 worker 机执行。
+- 环境名称:配置运行脚本的环境。
+- 失败重试次数:任务失败重新提交的次数。
+- 失败重试间隔:任务失败重新提交任务的时间间隔,以分为单位。
+- 超时警告:勾选超时警告、超时失败,当任务超过“超时时长”后,会发送告警邮件并且任务执行失败。
+- 添加依赖:需要判断的依赖任务,可以是某一个项目中的工作流具体的任务执行情况。
+- 前置任务:选择当前任务的前置任务,会将被选择的前置任务设置为当前任务的上游。
 
- <p align="center">
-   <img src="/img/depend-node2.png" width="80%" />
- </p>
+## 任务样例
 
-  > 假如,周报A同时还需要自身在上周二执行成功:
+Dependent 节点提供了逻辑判断功能,可以按照逻辑来检测所依赖节点的执行情况。
 
- <p align="center">
-   <img src="/img/depend-node3.png" width="80%" />
- </p>
\ No newline at end of file
+例如,A 流程为周报任务,B、C 流程为天任务,A 任务需要 B、C 任务在上周的每一天都执行成功,如图示:
+
+![dependent_task01](/img/tasks/demo/dependent_task01.png)
+
+例如,A 流程为周报任务,B、C 流程为天任务,A 任务需要 B 或 C 任务在上周的每一天都执行成功,如图示:
+
+![dependent_task02](/img/tasks/demo/dependent_task02.png)
+
+假如,周报 A 同时还需要自身在上周二执行成功:
+
+![dependent_task03](/img/tasks/demo/dependent_task03.png)
diff --git a/docs/img/depend-node-en.png b/docs/img/depend-node-en.png
deleted file mode 100644
index d1711d7c8b..0000000000
Binary files a/docs/img/depend-node-en.png and /dev/null differ
diff --git a/docs/img/depend-node.png b/docs/img/depend-node.png
deleted file mode 100644
index 5ea74bfccf..0000000000
Binary files a/docs/img/depend-node.png and /dev/null differ
diff --git a/docs/img/depend-node1-en.png b/docs/img/depend-node1-en.png
deleted file mode 100644
index 62314cb979..0000000000
Binary files a/docs/img/depend-node1-en.png and /dev/null differ
diff --git a/docs/img/depend-node2.png b/docs/img/depend-node2.png
deleted file mode 100644
index 7fcb1434df..0000000000
Binary files a/docs/img/depend-node2.png and /dev/null differ
diff --git a/docs/img/depend-node3-en.png b/docs/img/depend-node3-en.png
deleted file mode 100644
index 18ad849a6c..0000000000
Binary files a/docs/img/depend-node3-en.png and /dev/null differ
diff --git a/docs/img/depend-node3.png b/docs/img/depend-node3.png
deleted file mode 100644
index 88dceb2684..0000000000
Binary files a/docs/img/depend-node3.png and /dev/null differ
diff --git a/docs/img/dependent-nodes-en.png b/docs/img/dependent-nodes-en.png
deleted file mode 100644
index 152a55ea31..0000000000
Binary files a/docs/img/dependent-nodes-en.png and /dev/null differ
diff --git a/docs/img/dependent_edit.png b/docs/img/dependent_edit.png
deleted file mode 100644
index 1dac57468b..0000000000
Binary files a/docs/img/dependent_edit.png and /dev/null differ
diff --git a/docs/img/tasks/demo/dependent_task01.png b/docs/img/tasks/demo/dependent_task01.png
new file mode 100644
index 0000000000..68a140a15d
Binary files /dev/null and b/docs/img/tasks/demo/dependent_task01.png differ
diff --git a/docs/img/tasks/demo/dependent_task02.png b/docs/img/tasks/demo/dependent_task02.png
new file mode 100644
index 0000000000..3f2afa4158
Binary files /dev/null and b/docs/img/tasks/demo/dependent_task02.png differ
diff --git a/docs/img/tasks/demo/dependent_task03.png b/docs/img/tasks/demo/dependent_task03.png
new file mode 100644
index 0000000000..1bc2f50aac
Binary files /dev/null and b/docs/img/tasks/demo/dependent_task03.png differ
diff --git a/docs/img/tasks/icons/dependent.png b/docs/img/tasks/icons/dependent.png
new file mode 100644
index 0000000000..3f0b732c40
Binary files /dev/null and b/docs/img/tasks/icons/dependent.png differ
diff --git a/docs/img/tasks/icons/python.png b/docs/img/tasks/icons/python.png
index 9775b28be2..8bc4d516aa 100644
Binary files a/docs/img/tasks/icons/python.png and b/docs/img/tasks/icons/python.png differ


[dolphinscheduler] 04/15: [Fix][UI] Fix the task name validator error. (#10008)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit ca0f41c10f57b2c42260efe3a1813144768b8e68
Author: Amy0104 <97...@users.noreply.github.com>
AuthorDate: Thu May 12 15:27:55 2022 +0800

    [Fix][UI] Fix the task name validator error. (#10008)
    
    
    (cherry picked from commit 00f102959b5a07f56cb2d6f8973949cc4eee22b0)
---
 .../projects/task/components/node/fields/use-dependent.ts    | 12 +++++++-----
 .../src/views/projects/task/components/node/types.ts         |  4 +++-
 2 files changed, 10 insertions(+), 6 deletions(-)

diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-dependent.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-dependent.ts
index 1a149484f9..c650a815f0 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-dependent.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-dependent.ts
@@ -31,7 +31,8 @@ import type {
   IJsonItem,
   IDependpendItem,
   IDependTask,
-  ITaskState
+  ITaskState,
+  IDateType
 } from '../types'
 
 export function useDependent(model: { [field: string]: any }): IJsonItem[] {
@@ -172,7 +173,7 @@ export function useDependent(model: { [field: string]: any }): IJsonItem[] {
         label: t('project.node.last_month_end')
       }
     ]
-  }
+  } as { [key in IDateType]: { value: string; label: string }[] }
 
   const getProjectList = async () => {
     const result = await queryAllProjectList()
@@ -338,8 +339,8 @@ export function useDependent(model: { [field: string]: any }): IJsonItem[] {
             rule: {
               required: true,
               trigger: ['input', 'blur'],
-              validator(validate: any, value: string) {
-                if (!value) {
+              validator(validate: any, value: number) {
+                if (!value && value !== 0) {
                   return Error(t('project.node.task_name_tips'))
                 }
               }
@@ -351,9 +352,10 @@ export function useDependent(model: { [field: string]: any }): IJsonItem[] {
             span: 10,
             name: t('project.node.cycle_time'),
             props: {
-              onUpdateValue: (value: 'month') => {
+              onUpdateValue: (value: IDateType) => {
                 model.dependTaskList[i].dependItemList[j].dateOptions =
                   DATE_LSIT[value]
+                model.dependTaskList[i].dependItemList[j].dateValue = null
               }
             },
             options: CYCLE_LIST,
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
index 368125f20f..6750d84fc9 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
@@ -37,6 +37,7 @@ type SourceType = 'MYSQL' | 'HDFS' | 'HIVE'
 type ModelType = 'import' | 'export'
 type RelationType = 'AND' | 'OR'
 type ITaskType = TaskType
+type IDateType = 'hour' | 'day' | 'week' | 'month'
 
 interface IOption {
   label: string
@@ -383,5 +384,6 @@ export {
   IJsonItem,
   FormRules,
   IJsonItemParams,
-  IResponseJsonItem
+  IResponseJsonItem,
+  IDateType
 }


[dolphinscheduler] 05/15: [Fix-9975] The selected task instance was recreated when the Master service fail… (#9976)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 7b0e6fe5ecb56c5231bf0fe60d5253593df69b9d
Author: WangJPLeo <10...@users.noreply.github.com>
AuthorDate: Thu May 12 16:31:53 2022 +0800

    [Fix-9975] The selected task instance was recreated when the Master service fail… (#9976)
    
    * The selected task instance was recreated when the Master service failed over.
    
    * Returns the expression result directly.
    
    * Use Recovery to determine whether to use the old task instance.
    
    (cherry picked from commit dbdbfeaeee4b6aaaa738ef642a908f700f0cb9bc)
---
 .../dolphinscheduler/server/master/runner/WorkflowExecuteThread.java   | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
index 98556dc0f0..3b7b5511ae 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
@@ -1960,6 +1960,9 @@ public class WorkflowExecuteThread {
     private boolean isNewProcessInstance() {
         if (ExecutionStatus.RUNNING_EXECUTION == processInstance.getState() && processInstance.getRunTimes() == 1) {
             return true;
+        } else if (processInstance.getRecovery().equals(Flag.YES)) {
+            // host is empty use old task instance
+            return false;
         } else {
             return false;
         }


[dolphinscheduler] 12/15: [doc] Correct docs of development-environment-setup (#9995)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit ea284526f9c8e0e4a728243942f3343642a37be3
Author: chuxing <92...@qq.com>
AuthorDate: Mon May 16 09:53:45 2022 +0800

    [doc] Correct docs of development-environment-setup (#9995)
    
    
    (cherry picked from commit f666c64946ea100c59199940af29dcfeeb688379)
---
 .../development/development-environment-setup.md   | 43 ++++++++-----------
 .../development/development-environment-setup.md   | 49 +++++++++-------------
 2 files changed, 37 insertions(+), 55 deletions(-)

diff --git a/docs/docs/en/development/development-environment-setup.md b/docs/docs/en/development/development-environment-setup.md
index a270b8e5c4..15babb7210 100644
--- a/docs/docs/en/development/development-environment-setup.md
+++ b/docs/docs/en/development/development-environment-setup.md
@@ -5,10 +5,9 @@
 Before setting up the DolphinScheduler development environment, please make sure you have installed the software as below:
 
 * [Git](https://git-scm.com/downloads): DolphinScheduler version control system
-* [JDK](https://www.oracle.com/technetwork/java/javase/downloads/index.html): DolphinScheduler backend language
+* [JDK](https://www.oracle.com/technetwork/java/javase/downloads/index.html): DolphinScheduler backend language, have to use version after JDK 1.8
 * [Maven](http://maven.apache.org/download.cgi): Java Package Management System
-* [Node](https://nodejs.org/en/download): DolphinScheduler frontend
- language
+* [Node](https://nodejs.org/en/download): DolphinScheduler frontend, have to use version after Node 12.20.2
 
 ### Clone Git Repository
 
@@ -19,13 +18,11 @@ mkdir dolphinscheduler
 cd dolphinscheduler
 git clone git@github.com:apache/dolphinscheduler.git
 ```
-### compile source code
 
-i. If you use MySQL database, pay attention to modify pom.xml in the root project, and change the scope of the mysql-connector-java dependency to compile.
+### compile source code
 
 ii. Run `mvn clean install -Prelease -Dmaven.test.skip=true`
 
-
 ## Notice
 
 There are two ways to configure the DolphinScheduler development environment, standalone mode and normal mode
@@ -52,7 +49,8 @@ Find the class `org.apache.dolphinscheduler.server.StandaloneServer` in Intellij
 
 ### Start frontend server
 
-Install frontend dependencies and run it
+Install frontend dependencies and run it.
+> Note: You can see more detail about the frontend setting in [frontend development](./frontend-development.md).
 
 ```shell
 cd dolphinscheduler-ui
@@ -94,36 +92,29 @@ Following steps will guide how to start the DolphinScheduler backend service
 ##### Backend Start Prepare
 
 * Open project: Use IDE open the project, here we use Intellij IDEA as an example, after opening it will take a while for Intellij IDEA to complete the dependent download
-* Plugin installation(**Only required for 2.0 or later**)
 
- * Registry plug-in configuration, take Zookeeper as an example (registry.properties)
-  dolphinscheduler-service/src/main/resources/registry.properties
-  ```registry.properties
-   registry.plugin.name=zookeeper
-   registry.servers=127.0.0.1:2181
-  ```
 * File change
   * If you use MySQL as your metadata database, you need to modify `dolphinscheduler/pom.xml` and change the `scope` of the `mysql-connector-java` dependency to `compile`. This step is not necessary to use PostgreSQL
-  * Modify database configuration, modify the database configuration in the `dolphinscheduler-dao/src/main/resources/application-mysql.yaml`
+  * Modify database configuration, modify the database configuration in the `dolphinscheduler-master/src/main/resources/application.yaml`
+  * Modify database configuration, modify the database configuration in the `dolphinscheduler-worker/src/main/resources/application.yaml`
+  * Modify database configuration, modify the database configuration in the `dolphinscheduler-api/src/main/resources/application.yaml`
 
 
-  We here use MySQL with database, username, password named dolphinscheduler as an example
-  ```application-mysql.yaml
+We here use MySQL with database, username, password named dolphinscheduler as an example
+  ```application.yaml
    spring:
      datasource:
        driver-class-name: com.mysql.jdbc.Driver
        url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8
-       username: ds_user
+       username: dolphinscheduler
        password: dolphinscheduler
   ```
 
 * Log level: add a line `<appender-ref ref="STDOUT"/>` to the following configuration to enable the log to be displayed on the command line
 
-  `dolphinscheduler-server/src/main/resources/logback-worker.xml`
-  
-  `dolphinscheduler-server/src/main/resources/logback-master.xml` 
-  
-  `dolphinscheduler-api/src/main/resources/logback-api.xml` 
+  `dolphinscheduler-master/src/main/resources/logback-spring.xml`
+  `dolphinscheduler-worker/src/main/resources/logback-spring.xml`
+  `dolphinscheduler-api/src/main/resources/logback-spring.xml`
 
   here we add the result after modify as below:
 
@@ -141,9 +132,9 @@ Following steps will guide how to start the DolphinScheduler backend service
 
 There are three services that need to be started, including MasterServer, WorkerServer, ApiApplicationServer.
 
-* MasterServer:Execute function `main` in the class `org.apache.dolphinscheduler.server.master.MasterServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-master.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
-* WorkerServer:Execute function `main` in the class `org.apache.dolphinscheduler.server.worker.WorkerServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-worker.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
-* ApiApplicationServer:Execute function `main` in the class `org.apache.dolphinscheduler.api.ApiApplicationServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-api.xml -Dspring.profiles.active=api,mysql`. After it started, you could find Open API documentation in http://localhost:12345/dolphinscheduler/doc.html
+* MasterServer:Execute function `main` in the class `org.apache.dolphinscheduler.server.master.MasterServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
+* WorkerServer:Execute function `main` in the class `org.apache.dolphinscheduler.server.worker.WorkerServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
+* ApiApplicationServer:Execute function `main` in the class `org.apache.dolphinscheduler.api.ApiApplicationServer` by Intellij IDEA, with the configuration *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Dspring.profiles.active=api,mysql`. After it started, you could find Open API documentation in http://localhost:12345/dolphinscheduler/doc.html
 
 > The `mysql` in the VM Options `-Dspring.profiles.active=mysql` means specified configuration file
 
diff --git a/docs/docs/zh/development/development-environment-setup.md b/docs/docs/zh/development/development-environment-setup.md
index f6dba9a6db..e315e89fb9 100644
--- a/docs/docs/zh/development/development-environment-setup.md
+++ b/docs/docs/zh/development/development-environment-setup.md
@@ -5,9 +5,9 @@
 在搭建 DolphinScheduler 开发环境之前请确保你已经安装一下软件
 
 * [Git](https://git-scm.com/downloads): 版本控制系统
-* [JDK](https://www.oracle.com/technetwork/java/javase/downloads/index.html): 后端开发
+* [JDK](https://www.oracle.com/technetwork/java/javase/downloads/index.html): 后端开发,必须使用JDK1.8及以后的版本
 * [Maven](http://maven.apache.org/download.cgi): Java包管理系统
-* [Node](https://nodejs.org/en/download): 前端开发
+* [Node](https://nodejs.org/en/download): 前端开发,必须使用Node12.20.2及以后的版本
 
 ### 克隆代码库
 
@@ -18,11 +18,10 @@ mkdir dolphinscheduler
 cd dolphinscheduler
 git clone git@github.com:apache/dolphinscheduler.git
 ```
-### 编译源码 
-* 如果使用MySQL数据库,请注意修改pom.xml, 添加 ` mysql-connector-java ` 依赖。
-* 运行 `mvn clean install -Prelease -Dmaven.test.skip=true`
 
+### 编译源码
 
+* 运行 `mvn clean install -Prelease -Dmaven.test.skip=true`
 
 ## 开发者须知
 
@@ -50,6 +49,7 @@ DolphinScheduler 开发环境配置有两个方式,分别是standalone模式
 ### 启动前端
 
 安装前端依赖并运行前端组件
+> 注意:你可以在[frontend development](./frontend-development.md)里查看更多前端的相关配置
 
 ```shell
 cd dolphinscheduler-ui
@@ -90,38 +90,29 @@ DolphinScheduler 的元数据存储在关系型数据库中,目前支持的关
 ##### 必要的准备工作
 
 * 打开项目:使用开发工具打开项目,这里以 Intellij IDEA 为例,打开后需要一段时间,让 Intellij IDEA 完成以依赖的下载
-  
-* 插件的配置(**仅 2.0 及以后的版本需要**):
-
-  * 注册中心插件配置, 以Zookeeper 为例 (registry.properties)
-  dolphinscheduler-service/src/main/resources/registry.properties
-  ```registry.properties
-   registry.plugin.name=zookeeper
-   registry.servers=127.0.0.1:2181
-  ```
+
 * 必要的修改
   * 如果使用 MySQL 作为元数据库,需要先修改 `dolphinscheduler/pom.xml`,将 `mysql-connector-java` 依赖的 `scope` 改为 `compile`,使用 PostgreSQL 则不需要
-  * 修改数据库配置,修改 `dolphinscheduler-dao/src/main/resources/application-mysql.yaml` 文件中的数据库配置
-
+  * 修改 Master 数据库配置,修改 `dolphinscheduler-master/src/main/resources/application.yaml` 文件中的数据库配置
+  * 修改 Worker 数据库配置,修改 `dolphinscheduler-worker/src/main/resources/application.yaml` 文件中的数据库配置
+  * 修改 Api 数据库配置,修改 `dolphinscheduler-api/src/main/resources/application.yaml` 文件中的数据库配置
 
   本样例以 MySQL 为例,其中数据库名为 dolphinscheduler,账户名密码均为 dolphinscheduler
-  ```application-mysql.yaml
+  ```application.yaml
    spring:
      datasource:
        driver-class-name: com.mysql.jdbc.Driver
        url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8
-       username: ds_user
+       username: dolphinscheduler
        password: dolphinscheduler
   ```
 
 * 修改日志级别:为以下配置增加一行内容 `<appender-ref ref="STDOUT"/>` 使日志能在命令行中显示
-  
-  `dolphinscheduler-server/src/main/resources/logback-worker.xml`
-  
-  `dolphinscheduler-server/src/main/resources/logback-master.xml`
-  
-  `dolphinscheduler-api/src/main/resources/logback-api.xml` 
-  
+
+  `dolphinscheduler-master/src/main/resources/logback-spring.xml`
+  `dolphinscheduler-worker/src/main/resources/logback-spring.xml`
+  `dolphinscheduler-api/src/main/resources/logback-spring.xml`
+
   修改后的结果如下:
 
   ```diff
@@ -136,9 +127,9 @@ DolphinScheduler 的元数据存储在关系型数据库中,目前支持的关
 
 我们需要启动三个服务,包括 MasterServer,WorkerServer,ApiApplicationServer
 
-* MasterServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.server.master.MasterServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-master.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
-* WorkerServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.server.worker.WorkerServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-worker.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
-* ApiApplicationServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.api.ApiApplicationServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-api.xml -Dspring.profiles.active=api,mysql`。启动完成可以浏览 Open API 文档,地址为 http://localhost:12345/dolphinscheduler/doc.html
+* MasterServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.server.master.MasterServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
+* WorkerServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.server.worker.WorkerServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Ddruid.mysql.usePingMethod=false -Dspring.profiles.active=mysql`
+* ApiApplicationServer:在 Intellij IDEA 中执行 `org.apache.dolphinscheduler.api.ApiApplicationServer` 中的 `main` 方法,并配置 *VM Options* `-Dlogging.config=classpath:logback-spring.xml -Dspring.profiles.active=api,mysql`。启动完成可以浏览 Open API 文档,地址为 http://localhost:12345/dolphinscheduler/doc.html
 
 > VM Options `-Dspring.profiles.active=mysql` 中 `mysql` 表示指定的配置文件
 
@@ -152,4 +143,4 @@ pnpm install
 pnpm run dev
 ```
 
-截止目前,前后端已成功运行起来,浏览器访问[http://localhost:3000](http://localhost:3000),并使用默认账户密码 **admin/dolphinscheduler123** 即可完成登录
+截止目前,前后端已成功运行起来,浏览器访问[http://localhost:3000](http://localhost:3000),并使用默认账户密码 **admin/dolphinscheduler123** 即可完成登录
\ No newline at end of file


[dolphinscheduler] 10/15: [fix-9991][worker]fix statement is closed before resultSet.getMetaData() (#10014)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 44c6a053e7b162c42bba064d828891b664db4b42
Author: chuxing <92...@qq.com>
AuthorDate: Sun May 15 10:24:31 2022 +0800

    [fix-9991][worker]fix statement is closed before resultSet.getMetaData() (#10014)
    
    
    (cherry picked from commit df04c4aef0956acf3ee948c418d9c546cbc8771c)
---
 .../dolphinscheduler/plugin/task/sql/SqlTask.java  | 22 ++++++----------------
 1 file changed, 6 insertions(+), 16 deletions(-)

diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java
index 3b0fa0e8c5..b3943ce0a1 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java
@@ -179,7 +179,6 @@ public class SqlTask extends AbstractTaskExecutor {
                                   List<SqlBinds> postStatementsBinds,
                                   List<String> createFuncs) throws Exception {
         Connection connection = null;
-        ResultSet resultSet = null;
         try {
 
             // create connection
@@ -197,8 +196,7 @@ public class SqlTask extends AbstractTaskExecutor {
             // decide whether to executeQuery or executeUpdate based on sqlType
             if (sqlParameters.getSqlType() == SqlType.QUERY.ordinal()) {
                 // query statements need to be convert to JsonArray and inserted into Alert to send
-                resultSet = executeQuery(connection, mainStatementsBinds.get(0), "main");
-                result = resultProcess(resultSet);
+                result = executeQuery(connection, mainStatementsBinds.get(0), "main");
             } else if (sqlParameters.getSqlType() == SqlType.NON_QUERY.ordinal()) {
                 // non query statement
                 String updateResult = executeUpdate(connection, mainStatementsBinds, "main");
@@ -213,7 +211,7 @@ public class SqlTask extends AbstractTaskExecutor {
             logger.error("execute sql error: {}", e.getMessage());
             throw e;
         } finally {
-            close(resultSet, connection);
+            close(connection);
         }
     }
 
@@ -292,10 +290,11 @@ public class SqlTask extends AbstractTaskExecutor {
         setTaskAlertInfo(taskAlertInfo);
     }
 
-    private ResultSet executeQuery(Connection connection, SqlBinds sqlBinds, String handlerType) throws Exception {
+    private String executeQuery(Connection connection, SqlBinds sqlBinds, String handlerType) throws Exception {
         try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBinds)) {
             logger.info("{} statement execute query, for sql: {}", handlerType, sqlBinds.getSql());
-            return statement.executeQuery();
+            ResultSet resultSet = statement.executeQuery();
+            return resultProcess(resultSet);
         }
     }
 
@@ -329,18 +328,9 @@ public class SqlTask extends AbstractTaskExecutor {
     /**
      * close jdbc resource
      *
-     * @param resultSet resultSet
      * @param connection connection
      */
-    private void close(ResultSet resultSet, Connection connection) {
-        if (resultSet != null) {
-            try {
-                resultSet.close();
-            } catch (SQLException e) {
-                logger.error("close result set error : {}", e.getMessage(), e);
-            }
-        }
-
+    private void close(Connection connection) {
         if (connection != null) {
             try {
                 connection.close();


[dolphinscheduler] 06/15: [Fix][UI] Support only one file upload on the file manage page. (#10007)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 2eb8c626e358abcf14bd597727a3d82c364dda1d
Author: Amy0104 <97...@users.noreply.github.com>
AuthorDate: Thu May 12 16:54:03 2022 +0800

    [Fix][UI] Support only one file upload on the file manage page. (#10007)
    
    
    (cherry picked from commit 18bfe6399d5faa69f84d7b673269a96f8082ead6)
---
 .../src/views/resource/file/upload/index.tsx             | 16 +++++++++++++++-
 .../src/views/resource/file/upload/use-form.ts           |  1 +
 2 files changed, 16 insertions(+), 1 deletion(-)

diff --git a/dolphinscheduler-ui/src/views/resource/file/upload/index.tsx b/dolphinscheduler-ui/src/views/resource/file/upload/index.tsx
index 035b5a715a..b2aee1fabe 100644
--- a/dolphinscheduler-ui/src/views/resource/file/upload/index.tsx
+++ b/dolphinscheduler-ui/src/views/resource/file/upload/index.tsx
@@ -44,16 +44,24 @@ export default defineComponent({
     const customRequest = ({ file }: any) => {
       state.uploadForm.name = file.name
       state.uploadForm.file = file.file
+      state.uploadFormNameRef.validate({
+        trigger: 'input'
+      })
     }
 
     const handleFile = () => {
       handleUploadFile(ctx.emit, hideModal, resetForm)
     }
 
+    const removeFile = () => {
+      state.uploadForm.name = ''
+    }
+
     return {
       hideModal,
       customRequest,
       handleFile,
+      removeFile,
       ...toRefs(state)
     }
   },
@@ -70,7 +78,11 @@ export default defineComponent({
         confirmLoading={this.saving}
       >
         <NForm rules={this.rules} ref='uploadFormRef'>
-          <NFormItem label={t('resource.file.file_name')} path='name'>
+          <NFormItem
+            label={t('resource.file.file_name')}
+            path='name'
+            ref='uploadFormNameRef'
+          >
             <NInput
               v-model={[this.uploadForm.name, 'value']}
               placeholder={t('resource.file.enter_name_tips')}
@@ -90,6 +102,8 @@ export default defineComponent({
               v-model={[this.uploadForm.file, 'value']}
               customRequest={this.customRequest}
               class='btn-upload'
+              max={1}
+              onRemove={this.removeFile}
             >
               <NButton>{t('resource.file.upload_files')}</NButton>
             </NUpload>
diff --git a/dolphinscheduler-ui/src/views/resource/file/upload/use-form.ts b/dolphinscheduler-ui/src/views/resource/file/upload/use-form.ts
index 58ec7dbe2b..6dff47db32 100644
--- a/dolphinscheduler-ui/src/views/resource/file/upload/use-form.ts
+++ b/dolphinscheduler-ui/src/views/resource/file/upload/use-form.ts
@@ -36,6 +36,7 @@ export function useForm() {
 
   const state = reactive({
     uploadFormRef: ref(),
+    uploadFormNameRef: ref(),
     uploadForm: defaultValue(),
     saving: false,
     rules: {


[dolphinscheduler] 02/15: [Fix-10002] Fix some bugs in datasource list (#10005)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 010d59f1d658e385cdf7538c28d9b8bdff9125ec
Author: rockfang <65...@qq.com>
AuthorDate: Thu May 12 14:11:52 2022 +0800

    [Fix-10002] Fix some bugs in datasource list (#10005)
    
    * fix: fix ellipsis bug in table column
    
    * fix ellipsis bug in table column
    
    * fix: json-highlight component support nested jsonstring
    
    * fix: make datasource description show
    
    (cherry picked from commit 53cd7009deae072c5284ea881204ed2e8845b572)
---
 .../src/views/datasource/list/json-highlight.tsx               | 10 +++++++---
 dolphinscheduler-ui/src/views/datasource/list/use-columns.ts   |  2 +-
 2 files changed, 8 insertions(+), 4 deletions(-)

diff --git a/dolphinscheduler-ui/src/views/datasource/list/json-highlight.tsx b/dolphinscheduler-ui/src/views/datasource/list/json-highlight.tsx
index 017702217c..26a3b67aa8 100644
--- a/dolphinscheduler-ui/src/views/datasource/list/json-highlight.tsx
+++ b/dolphinscheduler-ui/src/views/datasource/list/json-highlight.tsx
@@ -56,9 +56,13 @@ const syntaxHighlight = (json: string) => {
     lines.push(
       <NText tag='div' class={styles['line']}>
         <NText type='error'>"{key}": </NText>
-        <NText type={type}>
-          "{value}"{i !== len - 1 ? ',' : ''}
-        </NText>
+        {isPlainObject(value) ? (
+          syntaxHighlight(JSON.stringify(value))
+        ) : (
+          <NText type={type}>
+            "{value}"{i !== len - 1 ? ',' : ''}
+          </NText>
+        )}
       </NText>
     )
   }
diff --git a/dolphinscheduler-ui/src/views/datasource/list/use-columns.ts b/dolphinscheduler-ui/src/views/datasource/list/use-columns.ts
index 604e2fd720..c43d71fa23 100644
--- a/dolphinscheduler-ui/src/views/datasource/list/use-columns.ts
+++ b/dolphinscheduler-ui/src/views/datasource/list/use-columns.ts
@@ -83,7 +83,7 @@ export function useColumns(onCallback: Function) {
         title: t('datasource.description'),
         key: 'note',
         ...COLUMN_WIDTH_CONFIG['note'],
-        render: (rowData) => rowData.description || '-'
+        render: (rowData) => rowData.note || '-'
       },
       {
         title: t('datasource.create_time'),


[dolphinscheduler] 14/15: [Bug] fix run on docker and k8s (#10026)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhongjiajie pushed a commit to branch 3.0.0-beta-prepare
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git

commit 0f841d6ff94a05820e099251a16859dc947ef6e9
Author: caishunfeng <ca...@gmail.com>
AuthorDate: Mon May 16 15:41:45 2022 +0800

    [Bug] fix run on docker and k8s (#10026)
    
    * fix docker-compose init schema
    
    * recovery depend on zk
    
    * update doc and dockerfile
    
    * fix run on k8s
    
    * udpate doc
    
    * add DOCKER flag & update doc
    
    * remove repeat DOCKER env
    
    (cherry picked from commit b1885c7a26a5404ce42d8fb0f2d8533d9a7c83c1)
---
 deploy/docker/docker-compose.yml                   | 22 +++++++++++-----------
 .../deployment-dolphinscheduler-alert.yaml         |  2 ++
 .../templates/deployment-dolphinscheduler-api.yaml |  6 ++++--
 .../job-dolphinscheduler-schema-initializer.yaml   |  4 +++-
 .../statefulset-dolphinscheduler-master.yaml       |  2 ++
 .../statefulset-dolphinscheduler-worker.yaml       |  2 ++
 docs/docs/en/guide/start/docker.md                 | 17 +++++++++++++----
 docs/docs/zh/guide/start/docker.md                 | 17 +++++++++++++----
 .../src/main/bin/upgrade-schema.sh                 |  4 +++-
 dolphinscheduler-tools/src/main/docker/Dockerfile  |  3 ++-
 10 files changed, 55 insertions(+), 24 deletions(-)

diff --git a/deploy/docker/docker-compose.yml b/deploy/docker/docker-compose.yml
index 2c557c4dd7..032d4208a6 100644
--- a/deploy/docker/docker-compose.yml
+++ b/deploy/docker/docker-compose.yml
@@ -19,6 +19,9 @@ version: "3.8"
 services:
   dolphinscheduler-postgresql:
     image: bitnami/postgresql:11.11.0
+    ports:
+      - "5432:5432"
+    profiles: ["all", "schema"]
     environment:
       POSTGRESQL_USERNAME: root
       POSTGRESQL_PASSWORD: root
@@ -35,6 +38,7 @@ services:
 
   dolphinscheduler-zookeeper:
     image: bitnami/zookeeper:3.6.2
+    profiles: ["all"]
     environment:
       ALLOW_ANONYMOUS_LOGIN: "yes"
       ZOO_4LW_COMMANDS_WHITELIST: srvr,ruok,wchs,cons
@@ -51,7 +55,8 @@ services:
   dolphinscheduler-schema-initializer:
     image: ${HUB}/dolphinscheduler-tools:${TAG}
     env_file: .env
-    command: [ bin/create-schema.sh ]
+    profiles: ["schema"]
+    command: [ tools/bin/upgrade-schema.sh ]
     depends_on:
       dolphinscheduler-postgresql:
         condition: service_healthy
@@ -66,15 +71,14 @@ services:
     image: ${HUB}/dolphinscheduler-api:${TAG}
     ports:
       - "12345:12345"
+    profiles: ["all"]
     env_file: .env
     healthcheck:
-      test: [ "CMD", "curl", "http://localhost:12345/actuator/health" ]
+      test: [ "CMD", "curl", "http://localhost:12345/dolphinscheduler/actuator/health" ]
       interval: 30s
       timeout: 5s
       retries: 3
     depends_on:
-      dolphinscheduler-schema-initializer:
-        condition: service_completed_successfully
       dolphinscheduler-zookeeper:
         condition: service_healthy
     volumes:
@@ -86,15 +90,13 @@ services:
 
   dolphinscheduler-alert:
     image: ${HUB}/dolphinscheduler-alert-server:${TAG}
+    profiles: ["all"]
     env_file: .env
     healthcheck:
       test: [ "CMD", "curl", "http://localhost:50053/actuator/health" ]
       interval: 30s
       timeout: 5s
       retries: 3
-    depends_on:
-      dolphinscheduler-schema-initializer:
-        condition: service_completed_successfully
     volumes:
       - dolphinscheduler-logs:/opt/dolphinscheduler/logs
     networks:
@@ -102,6 +104,7 @@ services:
 
   dolphinscheduler-master:
     image: ${HUB}/dolphinscheduler-master:${TAG}
+    profiles: ["all"]
     env_file: .env
     healthcheck:
       test: [ "CMD", "curl", "http://localhost:5679/actuator/health" ]
@@ -109,8 +112,6 @@ services:
       timeout: 5s
       retries: 3
     depends_on:
-      dolphinscheduler-schema-initializer:
-        condition: service_completed_successfully
       dolphinscheduler-zookeeper:
         condition: service_healthy
     volumes:
@@ -121,6 +122,7 @@ services:
 
   dolphinscheduler-worker:
     image: ${HUB}/dolphinscheduler-worker:${TAG}
+    profiles: ["all"]
     env_file: .env
     healthcheck:
       test: [ "CMD", "curl", "http://localhost:1235/actuator/health" ]
@@ -128,8 +130,6 @@ services:
       timeout: 5s
       retries: 3
     depends_on:
-      dolphinscheduler-schema-initializer:
-        condition: service_completed_successfully
       dolphinscheduler-zookeeper:
         condition: service_healthy
     volumes:
diff --git a/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-alert.yaml b/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-alert.yaml
index aba9e60d22..49ff69ac83 100644
--- a/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-alert.yaml
+++ b/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-alert.yaml
@@ -67,6 +67,8 @@ spec:
           env:
             - name: TZ
               value: {{ .Values.timezone }}
+            - name: SPRING_JACKSON_TIME_ZONE
+              value: {{ .Values.timezone }}
             {{- include "dolphinscheduler.database.env_vars" . | nindent 12 }}
             {{ range $key, $value :=  .Values.alert.env }}
             - name: {{ $key }}
diff --git a/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-api.yaml b/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-api.yaml
index 64ff6abd2e..f571f8c2fa 100644
--- a/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-api.yaml
+++ b/deploy/kubernetes/dolphinscheduler/templates/deployment-dolphinscheduler-api.yaml
@@ -65,6 +65,8 @@ spec:
           env:
             - name: TZ
               value: {{ .Values.timezone }}
+            - name: SPRING_JACKSON_TIME_ZONE
+              value: {{ .Values.timezone }}
             {{- include "dolphinscheduler.database.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.registry.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.fs_s3a.env_vars" . | nindent 12 }}
@@ -82,7 +84,7 @@ spec:
           {{- if .Values.api.livenessProbe.enabled }}
           livenessProbe:
             exec:
-              command: ["curl", "-s", "http://localhost:12345/actuator/health/liveness"]
+              command: ["curl", "-s", "http://localhost:12345/dolphinscheduler/actuator/health/liveness"]
             initialDelaySeconds: {{ .Values.api.livenessProbe.initialDelaySeconds }}
             periodSeconds: {{ .Values.api.livenessProbe.periodSeconds }}
             timeoutSeconds: {{ .Values.api.livenessProbe.timeoutSeconds }}
@@ -92,7 +94,7 @@ spec:
           {{- if .Values.api.readinessProbe.enabled }}
           readinessProbe:
             exec:
-              command: ["curl", "-s", "http://localhost:12345/actuator/health/readiness"]
+              command: ["curl", "-s", "http://localhost:12345/dolphinscheduler/actuator/health/readiness"]
             initialDelaySeconds: {{ .Values.api.readinessProbe.initialDelaySeconds }}
             periodSeconds: {{ .Values.api.readinessProbe.periodSeconds }}
             timeoutSeconds: {{ .Values.api.readinessProbe.timeoutSeconds }}
diff --git a/deploy/kubernetes/dolphinscheduler/templates/job-dolphinscheduler-schema-initializer.yaml b/deploy/kubernetes/dolphinscheduler/templates/job-dolphinscheduler-schema-initializer.yaml
index 4df2ebd3ec..bb5780d670 100644
--- a/deploy/kubernetes/dolphinscheduler/templates/job-dolphinscheduler-schema-initializer.yaml
+++ b/deploy/kubernetes/dolphinscheduler/templates/job-dolphinscheduler-schema-initializer.yaml
@@ -38,10 +38,12 @@ spec:
           image: {{ include "dolphinscheduler.image.fullname.tools" . }}
           imagePullPolicy: {{ .Values.image.pullPolicy }}
           args:
-            - bin/create-schema.sh
+            - tools/bin/upgrade-schema.sh
           env:
             - name: TZ
               value: {{ .Values.timezone }}
+            - name: SPRING_JACKSON_TIME_ZONE
+              value: {{ .Values.timezone }}
             {{- include "dolphinscheduler.database.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.registry.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.fs_s3a.env_vars" . | nindent 12 }}
diff --git a/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-master.yaml b/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-master.yaml
index 503ff23f84..f1f8f4b86e 100644
--- a/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-master.yaml
+++ b/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-master.yaml
@@ -62,6 +62,8 @@ spec:
           env:
             - name: TZ
               value: {{ .Values.timezone }}
+            - name: SPRING_JACKSON_TIME_ZONE
+              value: {{ .Values.timezone }}
             {{- include "dolphinscheduler.database.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.registry.env_vars" . | nindent 12 }}
             {{- include "dolphinscheduler.fs_s3a.env_vars" . | nindent 12 }}
diff --git a/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-worker.yaml b/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-worker.yaml
index d148f5122c..ebab09f279 100644
--- a/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-worker.yaml
+++ b/deploy/kubernetes/dolphinscheduler/templates/statefulset-dolphinscheduler-worker.yaml
@@ -62,6 +62,8 @@ spec:
           env:
             - name: TZ
               value: {{ .Values.timezone }}
+            - name: SPRING_JACKSON_TIME_ZONE
+              value: {{ .Values.timezone }}
             - name: ALERT_LISTEN_HOST
               value: {{ include "dolphinscheduler.fullname" . }}-alert
             {{- include "dolphinscheduler.database.env_vars" . | nindent 12 }}
diff --git a/docs/docs/en/guide/start/docker.md b/docs/docs/en/guide/start/docker.md
index db7d869628..87a3bb9bc9 100644
--- a/docs/docs/en/guide/start/docker.md
+++ b/docs/docs/en/guide/start/docker.md
@@ -9,7 +9,7 @@ or zookeeper server already exists.
 ## Prepare
 
 - [Docker](https://docs.docker.com/engine/install/) 1.13.1+
-- [Docker Compose](https://docs.docker.com/compose/) 1.11.0+
+- [Docker Compose](https://docs.docker.com/compose/) 1.28.0+
 
 ## Start Server
 
@@ -54,9 +54,13 @@ $ tar -zxf apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src.tar.gz
 # Going to docker-compose's location
 # For Mac or Linux users 
 $ cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src/deploy/docker
-# For Windows users
-$ cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src\deploy\docker
-$ docker-compose up -d
+# For Windows users, you should run command `cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src\deploy\docker`
+
+# Initialize the database, use profile schema
+$ docker-compose --profile schema up -d
+
+# start all dolphinscheduler server, use profile all
+$ docker-compose --profile all up -d
 ```
 
 > NOTES: It will not only start DolphinScheduler servers but also some others necessary services like PostgreSQL(with `root`
@@ -78,6 +82,7 @@ $ docker run -d --name dolphinscheduler-tools \
     -e SPRING_DATASOURCE_URL="jdbc:postgresql://localhost:5432/<DATABASE>" \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
+    --net host \
     apache/dolphinscheduler-tools:"${DOLPHINSCHEDULER_VERSION}" bin/create-schema.sh
 # Starting DolphinScheduler service
 $ docker run -d --name dolphinscheduler-master \
@@ -87,6 +92,7 @@ $ docker run -d --name dolphinscheduler-master \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-master:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-worker \
     -e DATABASE="postgresql" \
@@ -95,6 +101,7 @@ $ docker run -d --name dolphinscheduler-worker \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-worker:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-api \
     -e DATABASE="postgresql" \
@@ -103,6 +110,7 @@ $ docker run -d --name dolphinscheduler-api \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-api:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-alert-server \
     -e DATABASE="postgresql" \
@@ -111,6 +119,7 @@ $ docker run -d --name dolphinscheduler-alert-server \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-alert-server:"${DOLPHINSCHEDULER_VERSION}"
 ```
 
diff --git a/docs/docs/zh/guide/start/docker.md b/docs/docs/zh/guide/start/docker.md
index 8daa8fc497..cbaa8f9d89 100644
--- a/docs/docs/zh/guide/start/docker.md
+++ b/docs/docs/zh/guide/start/docker.md
@@ -7,7 +7,7 @@
 ## 前置条件
 
 - [Docker](https://docs.docker.com/engine/install/) 1.13.1+
-- [Docker Compose](https://docs.docker.com/compose/) 1.11.0+
+- [Docker Compose](https://docs.docker.com/compose/) 1.28.0+
 
 ## 启动服务
 
@@ -47,9 +47,13 @@ $ DOLPHINSCHEDULER_VERSION=3.0.0-beta-1
 $ tar -zxf apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src.tar.gz
 # Mac Linux 用户
 $ cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src/deploy/docker
-# Windows 用户
-$ cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src\deploy\docker
-$ docker-compose up -d
+#  Windows 用户, `cd apache-dolphinscheduler-"${DOLPHINSCHEDULER_VERSION}"-src\deploy\docker`
+
+# 如果需要初始化或者升级数据库结构,需要指定profile为schema
+$ docker-compose --profile schema up -d
+
+# 启动dolphinscheduler所有服务,指定profile为all
+$ docker-compose --profile all up -d
 ```
 
 > 提醒:通过 docker-compose 启动服务时,除了会启动 DolphinScheduler 对应的服务外,还会启动必要依赖服务,如数据库 PostgreSQL(用户 
@@ -70,6 +74,7 @@ $ docker run -d --name dolphinscheduler-tools \
     -e SPRING_DATASOURCE_URL="jdbc:postgresql://localhost:5432/<DATABASE>" \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
+    --net host \
     apache/dolphinscheduler-tools:"${DOLPHINSCHEDULER_VERSION}" bin/create-schema.sh
 # 启动 DolphinScheduler 对应的服务
 $ docker run -d --name dolphinscheduler-master \
@@ -79,6 +84,7 @@ $ docker run -d --name dolphinscheduler-master \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-master:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-worker \
     -e DATABASE="postgresql" \
@@ -87,6 +93,7 @@ $ docker run -d --name dolphinscheduler-worker \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-worker:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-api \
     -e DATABASE="postgresql" \
@@ -95,6 +102,7 @@ $ docker run -d --name dolphinscheduler-api \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-api:"${DOLPHINSCHEDULER_VERSION}"
 $ docker run -d --name dolphinscheduler-alert-server \
     -e DATABASE="postgresql" \
@@ -103,6 +111,7 @@ $ docker run -d --name dolphinscheduler-alert-server \
     -e SPRING_DATASOURCE_USERNAME="<USER>" \
     -e SPRING_DATASOURCE_PASSWORD="<PASSWORD>" \
     -e REGISTRY_ZOOKEEPER_CONNECT_STRING="localhost:2181" \
+    --net host \
     -d apache/dolphinscheduler-alert-server:"${DOLPHINSCHEDULER_VERSION}"
 ```
 
diff --git a/dolphinscheduler-tools/src/main/bin/upgrade-schema.sh b/dolphinscheduler-tools/src/main/bin/upgrade-schema.sh
index d11dc6a2c1..33d4e06db8 100755
--- a/dolphinscheduler-tools/src/main/bin/upgrade-schema.sh
+++ b/dolphinscheduler-tools/src/main/bin/upgrade-schema.sh
@@ -19,7 +19,9 @@
 BIN_DIR=$(dirname $0)
 DOLPHINSCHEDULER_HOME=${DOLPHINSCHEDULER_HOME:-$(cd $BIN_DIR/../..; pwd)}
 
-source "$DOLPHINSCHEDULER_HOME/bin/env/dolphinscheduler_env.sh"
+if [ "$DOCKER" != "true" ]; then
+  source "$DOLPHINSCHEDULER_HOME/bin/env/dolphinscheduler_env.sh"
+fi
 
 JAVA_OPTS=${JAVA_OPTS:-"-server -Duser.timezone=${SPRING_JACKSON_TIME_ZONE} -Xms1g -Xmx1g -Xmn512m -XX:+PrintGCDetails -Xloggc:gc.log -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=dump.hprof"}
 
diff --git a/dolphinscheduler-tools/src/main/docker/Dockerfile b/dolphinscheduler-tools/src/main/docker/Dockerfile
index 5c2fb45e45..811a8c2006 100644
--- a/dolphinscheduler-tools/src/main/docker/Dockerfile
+++ b/dolphinscheduler-tools/src/main/docker/Dockerfile
@@ -27,6 +27,7 @@ RUN apt update ; \
 
 WORKDIR $DOLPHINSCHEDULER_HOME
 
-ADD ./target/tools $DOLPHINSCHEDULER_HOME
+# see doc: https://dolphinscheduler.apache.org/en-us/docs/dev/user_doc/guide/upgrade.html
+ADD ./target/tools $DOLPHINSCHEDULER_HOME/tools
 
 ENTRYPOINT [ "/bin/bash" ]