You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by GitBox <gi...@apache.org> on 2022/08/23 09:33:44 UTC

[GitHub] [dolphinscheduler] jieguangzhou opened a new pull request, #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

jieguangzhou opened a new pull request, #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611

   <!--Thanks very much for contributing to Apache DolphinScheduler. Please review https://dolphinscheduler.apache.org/en-us/community/development/pull-request.html before opening a pull request.-->
   
   
   ## Purpose of the pull request
   
   close: #10995
   
   ## Brief change log
   
   - [x] YAML parse module
   - [x] YAML task examples
   - [ ] CLI
   - [ ] Document
   
   <!--*(for example:)*
     - *Add maven-checkstyle-plugin to root pom.xml*
   -->
   ## Verify this pull request
   
   <!--*(Please pick either of the following options)*-->
   
   This pull request is code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   <!--*(example:)*
     - *Added dolphinscheduler-dao tests for end-to-end.*
     - *Added CronUtilsTest to verify the change.*
     - *Manually verified the change by testing locally.* -->
   
   (or)
   
   If your pull request contain incompatible change, you should also add it to `docs/docs/en/guide/upgrede/incompatible.md`
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956552218


##########
dolphinscheduler-python/pydolphinscheduler/tests/core/test_yaml_process_define.py:
##########
@@ -0,0 +1,172 @@
+# 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.
+
+"""Test YAML process."""
+
+import os
+from unittest.mock import patch
+
+import pytest
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.yaml_process_define import (
+    ParseTool,
+    create_process_definition,
+    get_task_cls,
+)
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from tests.testing.task import Task
+
+
+def get_examples_folder():
+    """Get exmaple folder path for testing."""
+    base_folder = os.path.abspath(__file__)
+    examples_path = os.path.join(base_folder, "../../../examples/yaml_define")
+    return os.path.abspath(examples_path)

Review Comment:
   We have a convenient method to get the path, you can see more detail in https://github.com/apache/dolphinscheduler/blob/7a766cbcf2de82ea401ede11051b1059ef97919a/dolphinscheduler-python/pydolphinscheduler/tests/testing/path.py#L26



##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/yaml_process_define.py:
##########
@@ -0,0 +1,456 @@
+# 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.
+
+"""Parse YAML file to create process."""
+
+import logging
+import os
+import re
+from typing import Any, Dict
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.task import Task
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from pydolphinscheduler.utils.yaml_parser import YamlParser
+
+log = logging.getLogger(__file__)
+
+
+class ParseTool:
+    """Enhanced parsing tools."""
+
+    @staticmethod
+    def parse_string_param_if_file(string_param: str, **kwargs):
+        """Use $FILE{"data_path"} to load file from "data_path"."""
+        if string_param.startswith("$FILE"):
+            path = re.findall(r"\$FILE\{\"(.*?)\"\}", string_param)[0]
+            base_folder = kwargs.get("base_folder", ".")
+            path = ParseTool.get_possible_path(path, base_folder)
+            with open(path, "r") as read_file:
+                string_param = "".join(read_file)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_env(string_param: str, **kwargs):
+        """Use $ENV{env_name} to load environment variable "env_name"."""
+        if "$ENV" in string_param:
+            key = re.findall(r"\$ENV\{(.*?)\}", string_param)[0]
+            env_value = os.environ.get(key)
+            string_param = string_param.replace("$ENV{%s}" % key, env_value)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_config(string_param: str, **kwargs):
+        """Use ${CONFIG.var_name} to load variable "var_name" from configuration."""
+        if "${CONFIG" in string_param:
+            key = re.findall(r"\$\{CONFIG\.(.*?)\}", string_param)[0]
+            if hasattr(configuration, key):
+                string_param = getattr(configuration, key)
+            else:
+                string_param = configuration.get_single_config(key)
+
+        return string_param
+
+    @staticmethod
+    def get_possible_path(file_path, base_folder):
+        """Get file possible path.
+
+        Return new path if file_path is not exists, but base_folder + file_path exists
+        """
+        possible_path = file_path
+        if not os.path.exists(file_path):
+            new_path = os.path.join(base_folder, file_path)
+            if os.path.exists(new_path):
+                possible_path = new_path
+                print(f"{file_path} not exists, convert to {possible_path}")
+
+        return possible_path
+
+
+def get_task_cls(task_type) -> Task:
+    """Get the task class object by task_type (case compatible)."""
+    # only get task class from tasks.__all__
+    all_task_types = {type_.capitalize(): type_ for type_ in tasks.__all__}
+    task_type_cap = task_type.capitalize()
+    if task_type_cap not in all_task_types:
+        raise PyDSTaskNoFoundException(f"cant not find task {task_type}")
+
+    standard_name = all_task_types[task_type_cap]
+    return getattr(tasks, standard_name)
+
+
+class YamlProcess(YamlParser):
+    """Yaml parser for create process.
+
+    :param yaml_file: yaml file path.
+
+        examples1 ::
+
+            parser = YamlParser(yaml=...)
+            parser.create_process_definition()
+
+        examples2 ::
+
+            YamlParser(yaml=...).create_process_definition()
+
+    """
+
+    _parse_rules = [
+        ParseTool.parse_string_param_if_file,
+        ParseTool.parse_string_param_if_env,
+        ParseTool.parse_string_param_if_config,
+    ]
+
+    def __init__(self, yaml_file: str):
+        with open(yaml_file, "r") as f:
+            content = f.read()
+
+        self._base_folder = os.path.split(yaml_file)[0]
+        content = self.prepare_refer_process(content)
+        super().__init__(content)
+
+    def create_process_definition(self):
+        """Create process main function."""
+        # get process parameters with key "process"
+        process_params = self["process"]
+
+        # pop "run" parameter, used at the end
+        is_run = process_params.pop("run", False)
+
+        # use YamlProcess._parse_rules to parse special value of yaml file
+        process_params = self.parse_params(process_params)
+
+        process_name = process_params["name"]
+        print(f"Create Process: {process_name}")
+        with ProcessDefinition(**process_params) as pd:
+
+            # save dependencies between tasks
+            dependencies = {}
+
+            # save name and task mapping
+            name2task = {}
+
+            # get task datas with key "tasks"
+            for task_data in self["tasks"]:
+                task = self.parse_task(task_data, name2task)
+
+                deps = task_data.get("deps", [])
+                if deps:
+                    dependencies[task.name] = deps
+                name2task[task.name] = task
+
+            # build dependencies between task
+            for downstream_task_name, deps in dependencies.items():
+                downstream_task = name2task[downstream_task_name]
+                for upstram_task_name in deps:

Review Comment:
   typo here, should be  `upstream_task_name `



##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/yaml_process_define.py:
##########
@@ -0,0 +1,456 @@
+# 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.
+
+"""Parse YAML file to create process."""
+
+import logging
+import os
+import re
+from typing import Any, Dict
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.task import Task
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from pydolphinscheduler.utils.yaml_parser import YamlParser
+
+log = logging.getLogger(__file__)
+
+
+class ParseTool:
+    """Enhanced parsing tools."""
+
+    @staticmethod
+    def parse_string_param_if_file(string_param: str, **kwargs):
+        """Use $FILE{"data_path"} to load file from "data_path"."""
+        if string_param.startswith("$FILE"):
+            path = re.findall(r"\$FILE\{\"(.*?)\"\}", string_param)[0]
+            base_folder = kwargs.get("base_folder", ".")
+            path = ParseTool.get_possible_path(path, base_folder)
+            with open(path, "r") as read_file:
+                string_param = "".join(read_file)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_env(string_param: str, **kwargs):
+        """Use $ENV{env_name} to load environment variable "env_name"."""
+        if "$ENV" in string_param:
+            key = re.findall(r"\$ENV\{(.*?)\}", string_param)[0]
+            env_value = os.environ.get(key)
+            string_param = string_param.replace("$ENV{%s}" % key, env_value)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_config(string_param: str, **kwargs):
+        """Use ${CONFIG.var_name} to load variable "var_name" from configuration."""
+        if "${CONFIG" in string_param:
+            key = re.findall(r"\$\{CONFIG\.(.*?)\}", string_param)[0]
+            if hasattr(configuration, key):
+                string_param = getattr(configuration, key)
+            else:
+                string_param = configuration.get_single_config(key)
+
+        return string_param
+
+    @staticmethod
+    def get_possible_path(file_path, base_folder):
+        """Get file possible path.
+
+        Return new path if file_path is not exists, but base_folder + file_path exists
+        """
+        possible_path = file_path
+        if not os.path.exists(file_path):
+            new_path = os.path.join(base_folder, file_path)
+            if os.path.exists(new_path):
+                possible_path = new_path
+                print(f"{file_path} not exists, convert to {possible_path}")
+
+        return possible_path
+
+
+def get_task_cls(task_type) -> Task:
+    """Get the task class object by task_type (case compatible)."""
+    # only get task class from tasks.__all__
+    all_task_types = {type_.capitalize(): type_ for type_ in tasks.__all__}
+    task_type_cap = task_type.capitalize()
+    if task_type_cap not in all_task_types:
+        raise PyDSTaskNoFoundException(f"cant not find task {task_type}")
+
+    standard_name = all_task_types[task_type_cap]
+    return getattr(tasks, standard_name)
+
+
+class YamlProcess(YamlParser):
+    """Yaml parser for create process.
+
+    :param yaml_file: yaml file path.
+
+        examples1 ::
+
+            parser = YamlParser(yaml=...)
+            parser.create_process_definition()
+
+        examples2 ::
+
+            YamlParser(yaml=...).create_process_definition()
+
+    """
+
+    _parse_rules = [
+        ParseTool.parse_string_param_if_file,
+        ParseTool.parse_string_param_if_env,
+        ParseTool.parse_string_param_if_config,
+    ]
+
+    def __init__(self, yaml_file: str):
+        with open(yaml_file, "r") as f:
+            content = f.read()
+
+        self._base_folder = os.path.split(yaml_file)[0]
+        content = self.prepare_refer_process(content)
+        super().__init__(content)
+
+    def create_process_definition(self):
+        """Create process main function."""
+        # get process parameters with key "process"
+        process_params = self["process"]
+
+        # pop "run" parameter, used at the end
+        is_run = process_params.pop("run", False)
+
+        # use YamlProcess._parse_rules to parse special value of yaml file
+        process_params = self.parse_params(process_params)
+
+        process_name = process_params["name"]
+        print(f"Create Process: {process_name}")
+        with ProcessDefinition(**process_params) as pd:
+
+            # save dependencies between tasks
+            dependencies = {}
+
+            # save name and task mapping
+            name2task = {}
+
+            # get task datas with key "tasks"
+            for task_data in self["tasks"]:
+                task = self.parse_task(task_data, name2task)
+
+                deps = task_data.get("deps", [])
+                if deps:
+                    dependencies[task.name] = deps
+                name2task[task.name] = task
+
+            # build dependencies between task
+            for downstream_task_name, deps in dependencies.items():
+                downstream_task = name2task[downstream_task_name]
+                for upstram_task_name in deps:
+                    upstram_task = name2task[upstram_task_name]
+                    upstram_task >> downstream_task
+
+            pd.submit()
+            # if set is_run, run the process after submit
+            if is_run:
+                log.info(f"run process: {pd}")
+                pd.run()
+
+        return process_name
+
+    def parse_params(self, params: Any):
+        """Recursively resolves the parameter values.
+
+        The function operates params only when it encounters a string; other types continue recursively.
+        """
+        if isinstance(params, str):
+            for parse_rule in self._parse_rules:
+                params_ = params
+                params = parse_rule(params, base_folder=self._base_folder)
+                if params_ != params:
+                    print(f"parse {params_} -> {params}")
+
+        elif isinstance(params, list):
+            for index in range(len(params)):
+                params[index] = self.parse_params(params[index])
+
+        elif isinstance(params, dict):
+            for key, value in params.items():
+                params[key] = self.parse_params(value)
+
+        return params
+
+    @classmethod
+    def parse(cls, yaml_file: str):
+        """Recursively resolves the parameter values.
+
+        The function operates params only when it encounters a string; other types continue recursively.
+        """
+        process_name = cls(yaml_file).create_process_definition()
+        return process_name
+
+    def prepare_refer_process(self, content):
+        """Allow YAML files to reference process derived from other YAML files."""
+        process_paths = re.findall(r"\$PROCESS\{\"(.*?)\"\}", content)
+        for process_path in process_paths:
+            print(
+                f"find special token {process_path}, load process form {process_path}"
+            )
+            possible_path = ParseTool.get_possible_path(process_path, self._base_folder)
+            process_name = YamlProcess.parse(possible_path)
+            content = content.replace('$PROCESS{"%s"}' % process_path, process_name)
+
+        return content
+
+    def parse_task(self, task_data: dict, name2task: Dict[str, Task]):
+        """Parse various types of tasks.
+
+        :param task_data: dict.
+                {
+                    "task_type": "Shell",
+                    "params": {"name": "shell_task", "command":"ehco hellp"}
+                }
+
+        :param name2task: Dict[str, Task]), mapping of task_name and task
+
+
+        Some task type have special parse func:
+            if task type is Switch, use parse_switch;
+            if task type is Condition, use parse_condition;
+            if task type is Dependent, use parse_dependent;
+            other, we pass all task_params as input to task class, like "task_cls(**task_params)".
+        """
+        task_type = task_data["task_type"]
+        task_params = task_data["params"]
+        task_cls = get_task_cls(task_type)
+
+        # use YamlProcess._parse_rules to parse special value of yaml file
+        task_params = self.parse_params(task_params)
+
+        if task_cls == tasks.Switch:
+            task = self.parse_switch(task_params, name2task)
+
+        elif task_cls == tasks.Condition:
+            task = self.parse_condition(task_params, name2task)
+
+        elif task_cls == tasks.Dependent:
+            task = self.parse_dependent(task_params, name2task)
+
+        else:
+            task = task_cls(**task_params)
+        print(task_type, task)

Review Comment:
   do we really need `print` here?



##########
dolphinscheduler-python/pydolphinscheduler/tests/core/test_yaml_process_define.py:
##########
@@ -0,0 +1,172 @@
+# 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.
+
+"""Test YAML process."""
+
+import os
+from unittest.mock import patch
+
+import pytest
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.yaml_process_define import (
+    ParseTool,
+    create_process_definition,
+    get_task_cls,
+)
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from tests.testing.task import Task
+
+
+def get_examples_folder():
+    """Get exmaple folder path for testing."""

Review Comment:
   ```suggestion
       """Get example folder path for testing."""
   ```



##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/yaml_process_define.py:
##########
@@ -0,0 +1,456 @@
+# 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.
+
+"""Parse YAML file to create process."""
+
+import logging
+import os
+import re
+from typing import Any, Dict
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.task import Task
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from pydolphinscheduler.utils.yaml_parser import YamlParser
+
+log = logging.getLogger(__file__)
+
+
+class ParseTool:
+    """Enhanced parsing tools."""
+
+    @staticmethod
+    def parse_string_param_if_file(string_param: str, **kwargs):
+        """Use $FILE{"data_path"} to load file from "data_path"."""
+        if string_param.startswith("$FILE"):
+            path = re.findall(r"\$FILE\{\"(.*?)\"\}", string_param)[0]
+            base_folder = kwargs.get("base_folder", ".")
+            path = ParseTool.get_possible_path(path, base_folder)
+            with open(path, "r") as read_file:
+                string_param = "".join(read_file)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_env(string_param: str, **kwargs):
+        """Use $ENV{env_name} to load environment variable "env_name"."""
+        if "$ENV" in string_param:
+            key = re.findall(r"\$ENV\{(.*?)\}", string_param)[0]
+            env_value = os.environ.get(key)
+            string_param = string_param.replace("$ENV{%s}" % key, env_value)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_config(string_param: str, **kwargs):
+        """Use ${CONFIG.var_name} to load variable "var_name" from configuration."""
+        if "${CONFIG" in string_param:
+            key = re.findall(r"\$\{CONFIG\.(.*?)\}", string_param)[0]
+            if hasattr(configuration, key):
+                string_param = getattr(configuration, key)
+            else:
+                string_param = configuration.get_single_config(key)
+
+        return string_param
+
+    @staticmethod
+    def get_possible_path(file_path, base_folder):
+        """Get file possible path.
+
+        Return new path if file_path is not exists, but base_folder + file_path exists
+        """
+        possible_path = file_path
+        if not os.path.exists(file_path):

Review Comment:
   can you use module `Pathlib` to handle the path issue instead of `os.path`?



##########
dolphinscheduler-python/pydolphinscheduler/tests/core/test_yaml_process_define.py:
##########
@@ -0,0 +1,172 @@
+# 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.
+
+"""Test YAML process."""
+
+import os
+from unittest.mock import patch
+
+import pytest
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.yaml_process_define import (
+    ParseTool,
+    create_process_definition,
+    get_task_cls,
+)
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from tests.testing.task import Task
+
+
+def get_examples_folder():
+    """Get exmaple folder path for testing."""
+    base_folder = os.path.abspath(__file__)
+    examples_path = os.path.join(base_folder, "../../../examples/yaml_define")
+    return os.path.abspath(examples_path)
+
+
+@pytest.mark.parametrize(
+    "string_param, expect",
+    [
+        ("$ENV{PROJECT_NAME}", "~/pydolphinscheduler"),
+    ],
+)
+def test_parse_tool_env(string_param, expect):
+    """Test parsing the environment variable."""
+    os.environ["PROJECT_NAME"] = expect
+    assert expect == ParseTool.parse_string_param_if_env(string_param)

Review Comment:
   can you also add test for function `get_possible_path`?



##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/core/yaml_process_define.py:
##########
@@ -0,0 +1,456 @@
+# 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.
+
+"""Parse YAML file to create process."""
+
+import logging
+import os
+import re
+from typing import Any, Dict
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.task import Task
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from pydolphinscheduler.utils.yaml_parser import YamlParser
+
+log = logging.getLogger(__file__)
+
+
+class ParseTool:
+    """Enhanced parsing tools."""
+
+    @staticmethod
+    def parse_string_param_if_file(string_param: str, **kwargs):
+        """Use $FILE{"data_path"} to load file from "data_path"."""
+        if string_param.startswith("$FILE"):
+            path = re.findall(r"\$FILE\{\"(.*?)\"\}", string_param)[0]
+            base_folder = kwargs.get("base_folder", ".")
+            path = ParseTool.get_possible_path(path, base_folder)
+            with open(path, "r") as read_file:
+                string_param = "".join(read_file)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_env(string_param: str, **kwargs):
+        """Use $ENV{env_name} to load environment variable "env_name"."""
+        if "$ENV" in string_param:
+            key = re.findall(r"\$ENV\{(.*?)\}", string_param)[0]
+            env_value = os.environ.get(key)
+            string_param = string_param.replace("$ENV{%s}" % key, env_value)
+        return string_param
+
+    @staticmethod
+    def parse_string_param_if_config(string_param: str, **kwargs):
+        """Use ${CONFIG.var_name} to load variable "var_name" from configuration."""
+        if "${CONFIG" in string_param:
+            key = re.findall(r"\$\{CONFIG\.(.*?)\}", string_param)[0]
+            if hasattr(configuration, key):
+                string_param = getattr(configuration, key)
+            else:
+                string_param = configuration.get_single_config(key)
+
+        return string_param
+
+    @staticmethod
+    def get_possible_path(file_path, base_folder):
+        """Get file possible path.
+
+        Return new path if file_path is not exists, but base_folder + file_path exists
+        """
+        possible_path = file_path
+        if not os.path.exists(file_path):
+            new_path = os.path.join(base_folder, file_path)
+            if os.path.exists(new_path):
+                possible_path = new_path
+                print(f"{file_path} not exists, convert to {possible_path}")
+
+        return possible_path
+
+
+def get_task_cls(task_type) -> Task:
+    """Get the task class object by task_type (case compatible)."""
+    # only get task class from tasks.__all__
+    all_task_types = {type_.capitalize(): type_ for type_ in tasks.__all__}
+    task_type_cap = task_type.capitalize()
+    if task_type_cap not in all_task_types:
+        raise PyDSTaskNoFoundException(f"cant not find task {task_type}")
+
+    standard_name = all_task_types[task_type_cap]
+    return getattr(tasks, standard_name)
+
+
+class YamlProcess(YamlParser):
+    """Yaml parser for create process.
+
+    :param yaml_file: yaml file path.
+
+        examples1 ::
+
+            parser = YamlParser(yaml=...)
+            parser.create_process_definition()
+
+        examples2 ::
+
+            YamlParser(yaml=...).create_process_definition()
+
+    """
+
+    _parse_rules = [
+        ParseTool.parse_string_param_if_file,
+        ParseTool.parse_string_param_if_env,
+        ParseTool.parse_string_param_if_config,
+    ]
+
+    def __init__(self, yaml_file: str):
+        with open(yaml_file, "r") as f:
+            content = f.read()
+
+        self._base_folder = os.path.split(yaml_file)[0]
+        content = self.prepare_refer_process(content)
+        super().__init__(content)
+
+    def create_process_definition(self):
+        """Create process main function."""
+        # get process parameters with key "process"
+        process_params = self["process"]
+
+        # pop "run" parameter, used at the end
+        is_run = process_params.pop("run", False)
+
+        # use YamlProcess._parse_rules to parse special value of yaml file
+        process_params = self.parse_params(process_params)
+
+        process_name = process_params["name"]
+        print(f"Create Process: {process_name}")
+        with ProcessDefinition(**process_params) as pd:
+
+            # save dependencies between tasks
+            dependencies = {}
+
+            # save name and task mapping
+            name2task = {}
+
+            # get task datas with key "tasks"
+            for task_data in self["tasks"]:
+                task = self.parse_task(task_data, name2task)
+
+                deps = task_data.get("deps", [])
+                if deps:
+                    dependencies[task.name] = deps
+                name2task[task.name] = task
+
+            # build dependencies between task
+            for downstream_task_name, deps in dependencies.items():
+                downstream_task = name2task[downstream_task_name]
+                for upstram_task_name in deps:
+                    upstram_task = name2task[upstram_task_name]
+                    upstram_task >> downstream_task
+
+            pd.submit()
+            # if set is_run, run the process after submit
+            if is_run:
+                log.info(f"run process: {pd}")
+                pd.run()
+
+        return process_name
+
+    def parse_params(self, params: Any):
+        """Recursively resolves the parameter values.
+
+        The function operates params only when it encounters a string; other types continue recursively.
+        """
+        if isinstance(params, str):
+            for parse_rule in self._parse_rules:
+                params_ = params
+                params = parse_rule(params, base_folder=self._base_folder)
+                if params_ != params:
+                    print(f"parse {params_} -> {params}")
+
+        elif isinstance(params, list):
+            for index in range(len(params)):
+                params[index] = self.parse_params(params[index])
+
+        elif isinstance(params, dict):
+            for key, value in params.items():
+                params[key] = self.parse_params(value)
+
+        return params
+
+    @classmethod
+    def parse(cls, yaml_file: str):
+        """Recursively resolves the parameter values.
+
+        The function operates params only when it encounters a string; other types continue recursively.
+        """
+        process_name = cls(yaml_file).create_process_definition()
+        return process_name
+
+    def prepare_refer_process(self, content):
+        """Allow YAML files to reference process derived from other YAML files."""
+        process_paths = re.findall(r"\$PROCESS\{\"(.*?)\"\}", content)
+        for process_path in process_paths:
+            print(
+                f"find special token {process_path}, load process form {process_path}"
+            )
+            possible_path = ParseTool.get_possible_path(process_path, self._base_folder)
+            process_name = YamlProcess.parse(possible_path)
+            content = content.replace('$PROCESS{"%s"}' % process_path, process_name)
+
+        return content
+
+    def parse_task(self, task_data: dict, name2task: Dict[str, Task]):
+        """Parse various types of tasks.
+
+        :param task_data: dict.
+                {
+                    "task_type": "Shell",
+                    "params": {"name": "shell_task", "command":"ehco hellp"}
+                }
+
+        :param name2task: Dict[str, Task]), mapping of task_name and task
+
+
+        Some task type have special parse func:
+            if task type is Switch, use parse_switch;
+            if task type is Condition, use parse_condition;
+            if task type is Dependent, use parse_dependent;
+            other, we pass all task_params as input to task class, like "task_cls(**task_params)".
+        """
+        task_type = task_data["task_type"]
+        task_params = task_data["params"]
+        task_cls = get_task_cls(task_type)
+
+        # use YamlProcess._parse_rules to parse special value of yaml file
+        task_params = self.parse_params(task_params)
+
+        if task_cls == tasks.Switch:
+            task = self.parse_switch(task_params, name2task)
+
+        elif task_cls == tasks.Condition:
+            task = self.parse_condition(task_params, name2task)
+
+        elif task_cls == tasks.Dependent:
+            task = self.parse_dependent(task_params, name2task)
+
+        else:
+            task = task_cls(**task_params)
+        print(task_type, task)
+        return task
+
+    def parse_switch(self, task_params, name2task):
+        """Parse Switch Task.
+
+        This is an example Yaml fragment of task_params
+
+        name: switch
+        condition:
+          - ["${var} > 1", switch_child_1]
+          - switch_child_2
+        """
+        from pydolphinscheduler.tasks.switch import (
+            Branch,
+            Default,
+            Switch,
+            SwitchCondition,
+        )
+
+        condition_datas = task_params["condition"]
+        conditions = []
+        for condition_data in condition_datas:
+
+            # if condition_data is string, for example: switch_child_2, set it to Default branch
+            if isinstance(condition_data, str):
+                conditions.append(Default(task=name2task.get(condition_data)))
+
+            # if condition_data is list of 2 items, for example:
+            # ["${var} > 1", switch_child_1], set it to Branch
+            elif isinstance(condition_data, list) and len(condition_data) == 2:
+                cond, task_name = condition_data
+                conditions.append(Branch(cond, task=name2task.get(task_name)))
+            else:
+                raise Exception(f"cant not parse {condition_data}")
+
+        switch = Switch(
+            name=task_params["name"], condition=SwitchCondition(*conditions)
+        )
+        return switch
+
+    def parse_condition(self, task_params, name2task):
+        """Parse Condition Task.
+
+        This is an example Yaml fragment of task_params
+
+        name: condition
+        success_task: success_branch
+        failed_task: fail_branch
+        OP: AND
+        groups:
+          -
+            OP: AND
+            groups:
+              - [pre_task_1, true]
+              - [pre_task_2, true]
+              - [pre_task_3, false]
+          -
+            OP: AND
+            groups:
+              - [pre_task_1, false]
+              - [pre_task_2, true]
+              - [pre_task_3, true]
+
+        """
+        from pydolphinscheduler.tasks.condition import (
+            FAILURE,
+            SUCCESS,
+            And,
+            Condition,
+            Or,
+        )
+
+        def get_op_cls(op):
+            cls = None
+            if op.lower() == "and":
+                cls = And
+            elif op.lower() == "or":
+                cls = Or
+            else:
+                raise Exception(f"OP must be in And or Or, but get: {op}")

Review Comment:
   please change all `exception` to `Exception("OP must be in And or Or, but get: %s", op)` instead of f-strings, f-strings will always format the string, but `Exception("OP must be in And or Or, but get: %s", op)` will only format when exception raise



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956591700


##########
dolphinscheduler-python/pydolphinscheduler/docs/source/tutorial.rst:
##########
@@ -219,5 +219,155 @@ named "tutorial" or "tutorial_decorator". The task graph of workflow like below:
    :language: text
    :lines: 24-28
 
+Create Process Using YAML File
+------------------------------
+
+We can use pydolphinscheduler CLI to create process using YAML file
+
+.. code-block:: bash
+

Review Comment:
   good idea, I will change it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231067262

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957957863


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   No file changes when I run the command `./mvnw spotless:apply -pl dolphinscheduler-python`
   
   <img width="681" alt="image" src="https://user-images.githubusercontent.com/31528124/187338036-3a85ee34-ebe5-415d-901f-39614ef2a6ec.png">
   
   But if I add a space to the YAML File, the command above will fix the space. I am not sure if that means the YAML file has been formatted.
   
   @zhongjiajie @EricGao888 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956592037


##########
dolphinscheduler-python/pydolphinscheduler/tests/core/test_yaml_process_define.py:
##########
@@ -0,0 +1,172 @@
+# 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.
+
+"""Test YAML process."""
+
+import os
+from unittest.mock import patch
+
+import pytest
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.yaml_process_define import (
+    ParseTool,
+    create_process_definition,
+    get_task_cls,
+)
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from tests.testing.task import Task
+
+
+def get_examples_folder():
+    """Get exmaple folder path for testing."""
+    base_folder = os.path.abspath(__file__)
+    examples_path = os.path.join(base_folder, "../../../examples/yaml_define")
+    return os.path.abspath(examples_path)
+
+
+@pytest.mark.parametrize(
+    "string_param, expect",
+    [
+        ("$ENV{PROJECT_NAME}", "~/pydolphinscheduler"),
+    ],
+)
+def test_parse_tool_env(string_param, expect):
+    """Test parsing the environment variable."""
+    os.environ["PROJECT_NAME"] = expect
+    assert expect == ParseTool.parse_string_param_if_env(string_param)

Review Comment:
   I will add that
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1237246860

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] EricGao888 commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
EricGao888 commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957950877


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   @zhongjiajie @jieguangzhou I think so. `Spotless` supports everything of official `prettier` and formatting `yaml` is a basic function of `prettier`. See https://github.com/diffplug/spotless/tree/main/plugin-maven#prettier for an example of using `Spotless` to format `Typescript` with `prettier`. I think formatting `yaml` is the same.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231065533

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r958217923


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Dependent_External.yaml:
##########
@@ -0,0 +1,35 @@
+# 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.
+
+# Define the process
+process:
+  name: "task_dependent_external"
+
+# Define the tasks under the process
+tasks:
+  - {
+      "task_type": "Shell",
+      "params": { "name": "task_1", "command": "echo task 1" },
+    }

Review Comment:
   @caishunfeng  do you have time to take a look the yaml template, I know you have learn other workflow engine using yaml file



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r962945885


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/DataX.yaml:
##########
@@ -0,0 +1,33 @@
+# 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.
+
+# Define the workflow
+workflow:
+  name: "DataX"
+
+# Define the tasks under the workflow
+tasks:
+  - task_type: DataX
+    name: task

Review Comment:
   General LGTM, but can we move `name` in the top of declare? I think it make more sense because `name` is unique in single task, like
   
   ```yaml
    - name: task
      task_type: DataX
      datasource_name: db
      datatarget_name: db
      sql: show tables;
      target_table: table_test
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1229087578

   @zhongjiajie PTAL, thanks
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956551408


##########
dolphinscheduler-python/pydolphinscheduler/docs/source/tutorial.rst:
##########
@@ -219,5 +219,155 @@ named "tutorial" or "tutorial_decorator". The task graph of workflow like below:
    :language: text
    :lines: 24-28
 
+Create Process Using YAML File
+------------------------------
+
+We can use pydolphinscheduler CLI to create process using YAML file
+
+.. code-block:: bash
+

Review Comment:
   I think we should only add a tutorial exampel instead of all yaml in this file. you can see more detail in 
   
   ```rst
   .. tab:: Tradition
   .. tab:: Task Decorator
   ```
   
   in this file. Maybe you can add new `tab` named `Yaml File` or others.
   
   And then for the detail for the detail of task type like `shell`, `SQL` I prefer to add they in seperate directory, Or maybe we can add them to exists tasks dirrectory, like yaml shell into https://dolphinscheduler.apache.org/python/dev/tasks/shell.html WDYT



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957490749


##########
dolphinscheduler-python/pydolphinscheduler/docs/source/tutorial.rst:
##########
@@ -219,5 +219,155 @@ named "tutorial" or "tutorial_decorator". The task graph of workflow like below:
    :language: text
    :lines: 24-28
 
+Create Process Using YAML File
+------------------------------
+
+We can use pydolphinscheduler CLI to create process using YAML file
+
+.. code-block:: bash
+

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1236091613

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1225976095

   Please retry analysis of this Pull-Request directly on [SonarCloud](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1228648950

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1230580669

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r958252482


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,60 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_1", "command": "echo pre_task_1" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_2", "command": "echo pre_task_2" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_3", "command": "echo pre_task_3" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "success_branch", "command": "echo success_branch" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "fail_branch", "command": "echo fail_branch" },
+    }
+  - task_type: Condition
+    params:
+      name: condition
+      success_task: success_branch
+      failed_task: fail_branch
+      OP: AND
+      groups:
+        - OP: AND
+          groups:
+            - [pre_task_1, true]

Review Comment:
   we should also add some key like `task` for this make it more meanful



##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,60 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_1", "command": "echo pre_task_1" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_2", "command": "echo pre_task_2" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "pre_task_3", "command": "echo pre_task_3" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "success_branch", "command": "echo success_branch" },
+    }
+  - {
+      "task_type": "Shell",
+      "params": { "name": "fail_branch", "command": "echo fail_branch" },
+    }
+  - task_type: Condition
+    params:
+      name: condition

Review Comment:
   Maybe we should move `name` in the top level



##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Dependent.yaml:
##########
@@ -0,0 +1,76 @@
+# 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.
+
+process:
+  name: "Dependent"
+
+# Define the tasks under the process
+tasks:
+  - task_type: Dependent
+    params:
+      name: dependent
+      denpendence:
+      OP: AND
+      groups:
+        - OP: Or
+          groups:
+            - [project-pydolphin, task_dependent_external, task_1]
+            - [project-pydolphin, task_dependent_external, task_2]
+        - OP: And
+          groups:
+            - [

Review Comment:
   Also here, add meanful key to recognite



##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/MoreConfiguration.yaml:
##########
@@ -0,0 +1,41 @@
+# 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.
+
+# Define the process
+process:
+  name: "MoreConfiguration"
+  param:
+    n: 1
+
+# Define the tasks under the process
+tasks:
+  - task_type: Shell
+    params:
+      name: shell_0
+      description: "yaml define task"
+      flag: "YES"
+      command: |
+        echo "$ENV{HOME}"
+        echo "${n}"
+      task_priority: "HIGH"
+      delay_time: 20
+      fail_retry_times: 30
+      fail_retry_interval: 5
+      timeout_flag: "CLOSE"
+      timeout: 60
+      local_params:

Review Comment:
   I think we should move this parameter in the top level, instead of under the `param`



##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Switch.yaml:
##########
@@ -0,0 +1,41 @@
+# 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.
+
+# Define the process
+process:
+  name: "Switch"
+  param:
+    var: 1
+
+# Define the tasks under the process
+tasks:
+  - task_type: Shell
+    params:
+      name: switch_child_1
+      command: echo switch_child_1
+
+  - task_type: Shell
+    params:
+      name: switch_child_2
+      command: echo switch_child_2
+
+  - task_type: Switch
+    params:
+      name: switch
+      condition:
+        - ["${var} > 1", switch_child_1]
+        - switch_child_2

Review Comment:
   Add meanful key here, same as condition task



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie merged pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie merged PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1236477804

   @zhongjiajie PTAL, thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1224179949

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1228518072

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1228356875

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r958211379


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Dependent_External.yaml:
##########
@@ -0,0 +1,35 @@
+# 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.
+
+# Define the process
+process:
+  name: "task_dependent_external"
+
+# Define the tasks under the process
+tasks:
+  - {
+      "task_type": "Shell",
+      "params": { "name": "task_1", "command": "echo task 1" },
+    }

Review Comment:
   I think this yaml format is a little odd, can we find some better for the yaml like https://github.com/argoproj/argo-workflows/blob/master/examples/dag-daemon-task.yaml



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r958059904


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   @EricGao888 @zhongjiajie Thanks for your suggestion. I have used Spotless to format the YAML file.
   
   I created an issue to discuss the YAML file format #11699



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231087197

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231091227

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] EricGao888 commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
EricGao888 commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957966421


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   @jieguangzhou No, currently we haven't configured `Spotless` in DS to check and format `yaml`. To enable this feature, you need to add some configurations as I mentioned above in `pom.xml` for `Spotless`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956551590


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   Is there any way we can auto format the yaml? and add some check in our CI?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956551776


##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/cli/commands.py:
##########
@@ -90,3 +91,16 @@ def config(getter, setter, init) -> None:
         for key, val in setter:
             set_single_config(key, val)
         click.echo("Set configuration done.")
+
+
+@cli.command()
+@click.option(
+    "--yaml_file",
+    "-file",
+    required=True,
+    help="YAML file path",
+    type=click.Path(exists=True),
+)
+def yaml(yaml_file) -> None:
+    """Create process using YAML file."""

Review Comment:
   `Create process definition using YAML file.`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1225871607

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![63.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list) [63.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1225976091

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![63.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list) [63.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231229092

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957940709


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   @EricGao888 does spotless support yaml format?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1236091527

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1225876456

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![63.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list) [63.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956551819


##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/tasks/__init__.py:
##########
@@ -35,6 +35,7 @@
 __all__ = [
     "Condition",
     "DataX",
+    "CustomDataX",

Review Comment:
   Thanks for adding this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1230577858

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957943066


##########
dolphinscheduler-python/pydolphinscheduler/examples/yaml_define/Condition.yaml:
##########
@@ -0,0 +1,48 @@
+# 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.
+
+# Define the process
+process:
+  name: "Condition"
+
+# Define the tasks under the process
+tasks:
+  - {"task_type": "Shell", "params": {"name": "pre_task_1", "command": "echo pre_task_1"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_2", "command": "echo pre_task_2"}}
+  - {"task_type": "Shell", "params": {"name": "pre_task_3", "command": "echo pre_task_3"}}
+  - {"task_type": "Shell", "params": {"name": "success_branch", "command": "echo success_branch"}}
+  - {"task_type": "Shell", "params": {"name": "fail_branch", "command": "echo fail_branch"}}

Review Comment:
   I can see it support in spotless readme
   <img width="1006" alt="image" src="https://user-images.githubusercontent.com/15820530/187334014-7a175b9e-6d86-4f2e-ac03-455a1bb48ce3.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231281005

   @zhongjiajie PTAL, thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1231227933

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956551694


##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/cli/commands.py:
##########
@@ -90,3 +91,16 @@ def config(getter, setter, init) -> None:
         for key, val in setter:
             set_single_config(key, val)
         click.echo("Set configuration done.")
+
+
+@cli.command()
+@click.option(
+    "--yaml_file",

Review Comment:
   maybe `--yaml-file` is better for long, and `-f` is for short



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r957491165


##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/cli/commands.py:
##########
@@ -90,3 +91,16 @@ def config(getter, setter, init) -> None:
         for key, val in setter:
             set_single_config(key, val)
         click.echo("Set configuration done.")
+
+
+@cli.command()
+@click.option(
+    "--yaml_file",
+    "-file",
+    required=True,
+    help="YAML file path",
+    type=click.Path(exists=True),
+)
+def yaml(yaml_file) -> None:
+    """Create process using YAML file."""

Review Comment:
   done



##########
dolphinscheduler-python/pydolphinscheduler/src/pydolphinscheduler/cli/commands.py:
##########
@@ -90,3 +91,16 @@ def config(getter, setter, init) -> None:
         for key, val in setter:
             set_single_config(key, val)
         click.echo("Set configuration done.")
+
+
+@cli.command()
+@click.option(
+    "--yaml_file",

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] codecov-commenter commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1225863439

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/11611?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#11611](https://codecov.io/gh/apache/dolphinscheduler/pull/11611?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a30d878) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/1c15a7d60536f0b02f1f285a083600088fd54278?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1c15a7d) will **increase** coverage by `0.16%`.
   > The diff coverage is `37.50%`.
   
   > :exclamation: Current head a30d878 differs from pull request most recent head 13a1ac2. Consider uploading reports for the commit 13a1ac2 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##                dev   #11611      +/-   ##
   ============================================
   + Coverage     39.67%   39.83%   +0.16%     
   - Complexity     4696     4707      +11     
   ============================================
     Files           993      996       +3     
     Lines         37695    37742      +47     
     Branches       4199     4200       +1     
   ============================================
   + Hits          14954    15035      +81     
   + Misses        21173    21136      -37     
   - Partials       1568     1571       +3     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/11611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...nscheduler/api/configuration/AppConfiguration.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbmZpZ3VyYXRpb24vQXBwQ29uZmlndXJhdGlvbi5qYXZh) | `93.93% <ø> (-0.18%)` | :arrow_down: |
   | [...cheduler/api/controller/AccessTokenController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvQWNjZXNzVG9rZW5Db250cm9sbGVyLmphdmE=) | `87.50% <ø> (ø)` | |
   | [...eduler/api/controller/AccessTokenV2Controller.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvQWNjZXNzVG9rZW5WMkNvbnRyb2xsZXIuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...hinscheduler/api/controller/ClusterController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvQ2x1c3RlckNvbnRyb2xsZXIuamF2YQ==) | `89.47% <ø> (ø)` | |
   | [...scheduler/api/controller/DataSourceController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvRGF0YVNvdXJjZUNvbnRyb2xsZXIuamF2YQ==) | `29.03% <0.00%> (ø)` | |
   | [...inscheduler/api/controller/ExecutorController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvRXhlY3V0b3JDb250cm9sbGVyLmphdmE=) | `23.07% <0.00%> (-1.52%)` | :arrow_down: |
   | [...phinscheduler/api/controller/LoggerController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvTG9nZ2VyQ29udHJvbGxlci5qYXZh) | `6.66% <0.00%> (-1.03%)` | :arrow_down: |
   | [...lphinscheduler/api/controller/LoginController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvTG9naW5Db250cm9sbGVyLmphdmE=) | `68.18% <0.00%> (ø)` | |
   | [...uler/api/controller/ProcessInstanceController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvUHJvY2Vzc0luc3RhbmNlQ29udHJvbGxlci5qYXZh) | `70.21% <ø> (ø)` | |
   | [.../api/controller/ProcessTaskRelationController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvUHJvY2Vzc1Rhc2tSZWxhdGlvbkNvbnRyb2xsZXIuamF2YQ==) | `18.75% <0.00%> (-2.68%)` | :arrow_down: |
   | ... and [83 more](https://codecov.io/gh/apache/dolphinscheduler/pull/11611/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] jieguangzhou commented on a diff in pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
jieguangzhou commented on code in PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#discussion_r956591820


##########
dolphinscheduler-python/pydolphinscheduler/tests/core/test_yaml_process_define.py:
##########
@@ -0,0 +1,172 @@
+# 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.
+
+"""Test YAML process."""
+
+import os
+from unittest.mock import patch
+
+import pytest
+
+from pydolphinscheduler import configuration, tasks
+from pydolphinscheduler.core.process_definition import ProcessDefinition
+from pydolphinscheduler.core.yaml_process_define import (
+    ParseTool,
+    create_process_definition,
+    get_task_cls,
+)
+from pydolphinscheduler.exceptions import PyDSTaskNoFoundException
+from tests.testing.task import Task
+
+
+def get_examples_folder():
+    """Get exmaple folder path for testing."""
+    base_folder = os.path.abspath(__file__)
+    examples_path = os.path.join(base_folder, "../../../examples/yaml_define")
+    return os.path.abspath(examples_path)

Review Comment:
   I will fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1228355779

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [Draft][DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1228643018

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11611)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11611&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #11611: [DSIP-11][Feature][python] Use the pydolphinscheduler to dynamically generate workflows from the configuration file

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #11611:
URL: https://github.com/apache/dolphinscheduler/pull/11611#issuecomment-1237247100

   Please retry analysis of this Pull-Request directly on [SonarCloud](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11611).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org