You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/08/26 17:23:53 UTC

[GitHub] [airflow] casassg opened a new pull request #10587: [WIP] Start @dag decorator

casassg opened a new pull request #10587:
URL: https://github.com/apache/airflow/pull/10587


   As a user I want to easily wrap a function to be a dag (or TaskGroup whenever AIP-34 is approved).
   
   This introduces some parts of code for this to work, missing to make tests functional and polish code (did it in an afternoon).
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710335169


   It's a DAG using DAG context underneath really. So it should work fine with anything that used to work before. I would like to add a decorator for TaskGroup as well.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483430354



##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,61 @@
+# 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.
+
+from typing import Any, Dict, List, Union
+
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG  # pylint: disable=R0401
+from airflow.models.xcom import XCOM_RETURN_KEY
+from inspect import signature, Parameter
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG
+
+import functools
+
+class DagParam:
+    """
+    
+    """
+
+    def __init__(self, dag: DAG, name: str, default: Any):
+        dag.params[name] = default
+        self._name = name
+        self._default = default
+
+    def resolve(self, context: Dict) -> Any:
+        """
+        Pull XCom value for the existing arg. This method is run during ``op.execute()``
+        in respectable context.
+        """
+        return self.context.get('params', {}).get(self._name, self._default)
+
+
+def dag(*dag_args, **dag_kwargs):

Review comment:
       Missing docstring 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483430257



##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,61 @@
+# 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.
+
+from typing import Any, Dict, List, Union
+
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG  # pylint: disable=R0401
+from airflow.models.xcom import XCOM_RETURN_KEY
+from inspect import signature, Parameter
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG
+
+import functools
+
+class DagParam:
+    """
+    
+    """
+
+    def __init__(self, dag: DAG, name: str, default: Any):
+        dag.params[name] = default
+        self._name = name
+        self._default = default
+
+    def resolve(self, context: Dict) -> Any:
+        """
+        Pull XCom value for the existing arg. This method is run during ``op.execute()``
+        in respectable context.
+        """
+        return self.context.get('params', {}).get(self._name, self._default)
+
+
+def dag(*dag_args, **dag_kwargs):
+    def wrapper(f: Callable):
+        dag_sig = signature(DAG.__init__).bind(dag_id=f.__name__)
+        dag_sig = dag_sig.bind(*dag_args, **dag_kwargs)
+        @functools.wraps(f)
+        def factory(*args, **kwargs):
+            sig = signature(f).bind(*args, **kwargs).apply_defaults()
+            with DAG(*dag_sig.args, **dag_sig.kwargs) as dag:
+                f_kwargs = {}
+                for name, value in sig.arguments.items():
+                    f_kwargs[name] = dag.param(name, value)
+                f(**f_kwargs)
+            return dag
+        return factory
+    return wrapper

Review comment:
       ```suggestion
       return wrapper
   
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510918817



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.

Review comment:
       https://airflow.apache.org/docs/stable/concepts.html#context-manager




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-713178004


   [The Workflow run](https://github.com/apache/airflow/actions/runs/318665882) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-686962554


   Thanks @casassg  for the draft! I think we need to add docs to make sure that everyone knows what's going on. As discussed I'm happy to add more test and probably an example DAG so I can play with this feature. Will this be ok with you?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710688343


   Rolledback a change as I didn't understand well how DagRun could be changed it's configuration from trigger interfaces. I proposed #11599 to fill the UI gap I see here. 
   
   Added a few more unit tests.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710716958


   [The Workflow run](https://github.com/apache/airflow/actions/runs/311720324) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510922910



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parameterize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+You can also use the parameters on jinja templates by using the ``{{context.params}}`` dictionary.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in ``globals()``. Which means you need to make sure to have

Review comment:
       ```suggestion
   .. note::
   
     Note that Airflow will only load DAGs that appear in ``globals()``. Which means you need to make sure to have
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506656027



##########
File path: tests/models/test_dag.py
##########
@@ -1754,3 +1757,98 @@ def test_count_number_queries(self, tasks_count):
                 state=State.RUNNING,
                 execution_date=TEST_DATE,
             )
+
+
+class TestDagDecorator:
+    DEFAULT_ARGS = {
+        "owner": "test",
+        "depends_on_past": True,
+        "start_date": timezone.utcnow(),
+        "retries": 1,
+        "retry_delay": timedelta(minutes=1),
+    }
+    DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+    VALUE = 42
+
+    def setUp(self):
+        super().setUp()
+        self.operator = None
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_set_dag_id(self):
+        @dag_decorator('test', default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'test'
+
+    def test_arg_not_set_fail(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def noop_pipeline(value):
+            @task
+            def return_num(num):
+                return num
+
+            return_num(value)
+        with pytest.raises(TypeError):
+            noop_pipeline()

Review comment:
       I'm not sure what exactly we are testing 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483429637



##########
File path: airflow/models/dag.py
##########
@@ -549,6 +550,9 @@ def pickle_id(self):
     def pickle_id(self, value):
         self._pickle_id = value
 
+    def param(self, name, default):
+        return DagParam(self, name, default)

Review comment:
       ```suggestion
           return DagParam(dag=self, name=name, default=default)
   ```
   But no strong opinion here




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710533432


   [The Workflow run](https://github.com/apache/airflow/actions/runs/311351853) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-713010812


   Should we extend `dag_discovery_safe_mode` to check for `DAG` or `dag` in lower case? Given that with this new decorator, no `DAG` will be imported?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483430027



##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,61 @@
+# 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.
+
+from typing import Any, Dict, List, Union
+
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG  # pylint: disable=R0401
+from airflow.models.xcom import XCOM_RETURN_KEY
+from inspect import signature, Parameter
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG
+
+import functools
+
+class DagParam:
+    """
+    
+    """
+
+    def __init__(self, dag: DAG, name: str, default: Any):
+        dag.params[name] = default

Review comment:
       Hm, should we add some method like `dag.add_param`? 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506623798



##########
File path: tests/models/test_dagparam.py
##########
@@ -0,0 +1,80 @@
+import unittest
+from datetime import datetime, timedelta
+
+from airflow.jobs.scheduler_job import TI
+from airflow.models import DagRun
+from airflow.models.dag import DAG
+from airflow.models.dagparam import dag as dag_decorator
+from airflow.operators.python import (
+    PythonOperator,
+    task)
+from airflow.utils import timezone
+from airflow.utils.state import State
+from airflow.utils.types import DagRunType
+from tests.test_utils.config import conf_vars
+from airflow.utils.session import create_session
+from airflow.utils.state import State
+from airflow.utils.types import DagRunType
+from tests.test_utils.db import clear_db_runs
+
+DEFAULT_ARGS = {
+    "owner": "test",
+    "depends_on_past": True,
+    "start_date": timezone.utcnow(),
+    "retries": 1,
+    "retry_delay": timedelta(minutes=1),
+}
+DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+
+VALUE = 42
+
+
+class TestDagParamRuntime(unittest.TestCase):
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_xcom_pass_to_op(self):
+        with DAG(dag_id="test_xcom_pass_to_op", default_args=DEFAULT_ARGS) as dag:
+            value = dag.param('value', default=VALUE)
+            @task
+            def return_num(num):
+                return num
+
+            xcom_arg = return_num(value)
+
+        dr = dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        xcom_arg.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull() == VALUE
+
+
+class TestDagDecorator:
+    @conf_vars({("core", "executor"): "DebugExecutor"})
+    def test_xcom_pass_to_op(self):
+
+        @dag_decorator(default_args=DEFAULT_ARGS)
+        def test_pipeline(some_param, other_param=VALUE):

Review comment:
       Yup. I can take a first pass at adding documentation so this is clear

##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,85 @@
+# 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.
+
+from typing import Any, Dict, Callable
+
+from inspect import signature
+
+
+import functools
+
+
+class DagParam:
+    """
+    Class that represents a DAG run parameter.
+
+    It can be used to parametrized your dags.
+
+    **Example**:
+
+        with DAG(...) as dag:
+          EmailOperator(subject=dag.param('subject', 'Hi from Airflow!'))
+
+    This object can be used in legacy Operators via Jinja.
+
+    :param current_dag: Dag that will be used to pull the parameter from.
+    :type current_dag: airflow.models.dag.DAG
+    :param name: key value which is used to set the parameter
+    :type name: str
+    :param default: Default value used if no parameter was set.
+    :type default: Any
+    """
+
+    def __init__(self, current_dag, name: str, default: Any):
+        current_dag.params[name] = default
+        self._name = name
+        self._default = default
+
+    def resolve(self, context: Dict) -> Any:
+        """
+        Pull DagParam value from DagRun context. This method is run during ``op.execute()``
+        in respectable context.
+        """
+        return context.get('params', {}).get(self._name, self._default)
+
+
+def dag(*dag_args, **dag_kwargs):

Review comment:
       Moved and added the import




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-713077373


   [The Workflow run](https://github.com/apache/airflow/actions/runs/318347559) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-716214747


   Rebased from master @turbaszek 


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506639557



##########
File path: airflow/models/dag.py
##########
@@ -2187,6 +2199,33 @@ def calculate_dagrun_date_fields(
 
         log.info("Setting next_dagrun for %s to %s", dag.dag_id, self.next_dagrun)
 
+def dag(*dag_args, **dag_kwargs):
+    """
+    Python dag decorator. Wraps a function into an Airflow DAG.
+    Accepts kwargs for operator kwarg. Can be used to parametrize DAGs.
+
+    :param dag_args: Arguments for DAG object
+    :type dag_args: list
+    :param dag_kwargs: Kwargs for DAG object.
+    :type dag_kwargs: dict
+    """
+    def wrapper(f: Callable):
+        dag_sig = signature(DAG.__init__)
+        dag_sig = dag_sig.bind_partial(*dag_args, **dag_kwargs)
+
+        @functools.wraps(f)
+        def factory(*args, **kwargs):
+            f_sig = signature(f).bind(*args, **kwargs)
+            f_sig.apply_defaults()
+            with DAG(*dag_sig.args, dag_id=f.__name__, **dag_sig.kwargs) as dag_obj:
+                f_kwargs = {}
+                for name, value in f_sig.arguments.items():
+                    f_kwargs[name] = dag_obj.param(name, value)
+                f(**f_kwargs)
+            return dag_obj

Review comment:
       Can you add some comments describing what's going on here?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r505969872



##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,61 @@
+# 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.
+
+from typing import Any, Dict, List, Union
+
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG  # pylint: disable=R0401
+from airflow.models.xcom import XCOM_RETURN_KEY
+from inspect import signature, Parameter
+from airflow.exceptions import AirflowException
+from airflow.models.dag import DAG
+
+import functools
+
+class DagParam:
+    """
+    
+    """
+
+    def __init__(self, dag: DAG, name: str, default: Any):
+        dag.params[name] = default

Review comment:
       Tecnically all it matter is that it's on the dictionary. Not sure if it's worth adding an extra method.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510924557



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Right now it would have to be `DAG = your_dag_function` -- and you also need the string `airflow` in the dag files somewhere `return all(s in content for s in (b'DAG', b'airflow'))` from  https://github.com/apache/airflow/blob/5fc06a6ef9c136a4861bc2426aea768e9a18dcbb/airflow/utils/file.py#L192-L211




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil merged pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #10587:
URL: https://github.com/apache/airflow/pull/10587


   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506673787



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Thoughts? I'm not sure how to run the system dags to make sure it runs CC @turbaszek 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710408996


   DAG are exported to globals as they are returned. So it should work. Will move my example DAG to example DAGs and add it as an automated DAG. That should check that it gets picked up?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506649853



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,52 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. code-block:: python
+
+  from airflow.decorators import dag, task
+
+  @dag(default_args=default_args, schedule_interval=None)
+  def send_server_ip(email: 'example@example.com')
+
+    # Using default connection as it's set to httpbin.org by default
+    get_ip = SimpleHttpOperator(
+        task_id='get_ip', endpoint='get', method='GET', xcom_push=True
+    )
+
+    @task(multiple_outputs=True)
+    def prepare_email(raw_json: str) -> Dict[str, str]:
+      external_ip = json.loads(raw_json)['origin']
+      return {
+        'subject':f'Server connected from {external_ip}',
+        'body': f'Seems like today your server executing Airflow is connected from the external IP {external_ip}<br>'
+      }
+
+    email_info = prepare_email(get_ip.output)
+
+    send_email = EmailOperator(
+        task_id='send_email',
+        to=email,
+        subject=email_info['subject'],
+        html_content=email_info['body']
+    )
+
+  my_dag = send_server_ip()

Review comment:
       Fine by me. I will add as a todo. Prioritizing adding unit tests for now




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710399728


   @casassg have you tested if scheduler pick ups DAGs defined this way? 


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r508861073



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Apart from this warning it would be good to add something like:
   ```
   This will work:
   dag = yout_dag_function()
   but this will not:
   yout_dag_function()
   ```
   
   The system DAG should be runnable by `pytest tests/core/test_example_dags_system.py -k example_dag_decorator`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r508861073



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Apart from this warning it would be good to add something like:
   ```
   This will work:
   dag = yout_dag_function()
   but this will not:
   yout_dag_function()
   ```
   
   The system DAG should be runnable by `pytest tests/core/test_example_dags_system.py -k example_dag_decorator --system core`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r511145522



##########
File path: docs/concepts.rst
##########
@@ -58,10 +58,12 @@ arbitrary number of tasks. In general, each one should correspond to a single
 logical workflow.
 
 .. note:: When searching for DAGs, Airflow only considers Python files
-   that contain the strings "airflow" and "DAG" by default. To consider
+   that contain the strings "airflow" and "dag" by default. To consider

Review comment:
       Should we add a note here that this is now case-insensitive?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506657305



##########
File path: tests/models/test_dag.py
##########
@@ -1754,3 +1757,98 @@ def test_count_number_queries(self, tasks_count):
                 state=State.RUNNING,
                 execution_date=TEST_DATE,
             )
+
+
+class TestDagDecorator:
+    DEFAULT_ARGS = {
+        "owner": "test",
+        "depends_on_past": True,
+        "start_date": timezone.utcnow(),
+        "retries": 1,
+        "retry_delay": timedelta(minutes=1),
+    }
+    DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+    VALUE = 42
+
+    def setUp(self):
+        super().setUp()
+        self.operator = None
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_set_dag_id(self):
+        @dag_decorator('test', default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'test'
+
+    def test_arg_not_set_fail(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def noop_pipeline(value):
+            @task
+            def return_num(num):
+                return num
+
+            return_num(value)
+        with pytest.raises(TypeError):
+            noop_pipeline()
+
+    def test_dag_id_function_name(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'noop_pipeline'
+
+    def test_xcom_pass_to_op(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def xcom_pass_to_op(value=self.VALUE):
+            @task
+            def return_num(num):
+                return num
+
+            xcom_arg = return_num(value)
+            self.operator = xcom_arg.operator
+
+        dag = xcom_pass_to_op()
+
+        dr = dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=self.DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        self.operator.run(start_date=self.DEFAULT_DATE, end_date=self.DEFAULT_DATE)
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull() == self.VALUE
+
+    @conf_vars({("core", "executor"): "DebugExecutor"})
+    def test_xcom_pass_to_op(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def pipeline(some_param, other_param=self.VALUE):
+            @task
+            def some_task(param):
+                return param
+
+            @task
+            def another_task(param):
+                return param
+
+            some_task(some_param)
+            another_task(other_param)
+
+        d = pipeline(self.VALUE)
+
+        d.run()

Review comment:
       Should we add sanity test for `@dag` function with both `@task` decorated operator and a DummyOperator?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510905159



##########
File path: airflow/config_templates/config.yml
##########
@@ -318,7 +318,7 @@
       version_added: ~
       type: string
       example: ~
-      default: "False"
+      default: "True"

Review comment:
       Is this intentional?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-713078880


   [The Workflow run](https://github.com/apache/airflow/actions/runs/318349535) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483429415



##########
File path: airflow/models/dag.py
##########
@@ -549,6 +550,9 @@ def pickle_id(self):
     def pickle_id(self, value):
         self._pickle_id = value
 
+    def param(self, name, default):

Review comment:
       Can you please add docstring? Pylint will be sad once we start running it over this 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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506637517



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,52 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. code-block:: python
+
+  from airflow.decorators import dag, task
+
+  @dag(default_args=default_args, schedule_interval=None)
+  def send_server_ip(email: 'example@example.com')
+
+    # Using default connection as it's set to httpbin.org by default
+    get_ip = SimpleHttpOperator(
+        task_id='get_ip', endpoint='get', method='GET', xcom_push=True
+    )
+
+    @task(multiple_outputs=True)
+    def prepare_email(raw_json: str) -> Dict[str, str]:
+      external_ip = json.loads(raw_json)['origin']
+      return {
+        'subject':f'Server connected from {external_ip}',
+        'body': f'Seems like today your server executing Airflow is connected from the external IP {external_ip}<br>'
+      }
+
+    email_info = prepare_email(get_ip.output)
+
+    send_email = EmailOperator(
+        task_id='send_email',
+        to=email,
+        subject=email_info['subject'],
+        html_content=email_info['body']
+    )
+
+  my_dag = send_server_ip()

Review comment:
       I would be in favor of putting this in example dag and using:
   ```
   .. exampleinclude:: /../airflow/providers/google/cloud/example_dags/example_bigquery_operations.py
       :language: python
       :dedent: 4
       :start-after: [START howto_operator_bigquery_create_dataset]
       :end-before: [END howto_operator_bigquery_create_dataset]
   ```
   
   In this way we will have a DAG that can be used for that can be used for tests




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710697734


   [The Workflow run](https://github.com/apache/airflow/actions/runs/311631410) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506649427



##########
File path: airflow/models/dag.py
##########
@@ -2187,6 +2199,33 @@ def calculate_dagrun_date_fields(
 
         log.info("Setting next_dagrun for %s to %s", dag.dag_id, self.next_dagrun)
 
+def dag(*dag_args, **dag_kwargs):
+    """
+    Python dag decorator. Wraps a function into an Airflow DAG.
+    Accepts kwargs for operator kwarg. Can be used to parametrize DAGs.
+
+    :param dag_args: Arguments for DAG object
+    :type dag_args: list
+    :param dag_kwargs: Kwargs for DAG object.
+    :type dag_kwargs: dict
+    """
+    def wrapper(f: Callable):
+        dag_sig = signature(DAG.__init__)
+        dag_sig = dag_sig.bind_partial(*dag_args, **dag_kwargs)
+
+        @functools.wraps(f)
+        def factory(*args, **kwargs):
+            f_sig = signature(f).bind(*args, **kwargs)
+            f_sig.apply_defaults()
+            with DAG(*dag_sig.args, dag_id=f.__name__, **dag_sig.kwargs) as dag_obj:
+                f_kwargs = {}
+                for name, value in f_sig.arguments.items():
+                    f_kwargs[name] = dag_obj.param(name, value)
+                f(**f_kwargs)
+            return dag_obj

Review comment:
       Lmk if this is enough. Adding more unit tests as well




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510918411



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.

Review comment:
       ```suggestion
   In addition to creating DAGs using context manager, in Airflow 2.0 you can also create DAGs from a function.
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-716153000


   @casassg would you mind rebasing?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r483431075



##########
File path: tests/models/test_dagparam.py
##########
@@ -0,0 +1,64 @@
+from airflow.models.dag import DAG
+from airflow.utils.types import DagRunType
+from airflow.models.dagparam import dag
+from airflow.operators.python import (
+    PythonOperator
+)
+from tests.test_utils.config import conf_vars
+from datetime import datetime, timedelta, timezone
+
+DEFAULT_ARGS = {
+    "owner": "test",
+    "depends_on_past": True,
+    "start_date": datetime.today(),
+    "retries": 1,
+    "retry_delay": timedelta(minutes=1),
+}
+DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+
+VALUE = 42
+
+class TestDagParamRuntime:
+    def test_xcom_pass_to_op(self):
+        with DAG(dag_id="test_xcom_pass_to_op", default_args=DEFAULT_ARGS) as dag:
+            value = dag.param('value', default=VALUE)
+            operator = PythonOperator(
+                python_callable=id,
+                op_args=[value],
+                task_id="return_value_1",
+                do_xcom_push=True,
+            )
+        dr = dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull() == VALUE

Review comment:
       We should add tearDown to clear the side effect. Helper methods are in `tests.tests_utils.db`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510905395



##########
File path: airflow/config_templates/config.yml
##########
@@ -318,7 +318,7 @@
       version_added: ~
       type: string
       example: ~
-      default: "False"
+      default: "True"

Review comment:
       If yes, will need a note in Updating.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.

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



[GitHub] [airflow] ashb commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510923253



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parameterize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+You can also use the parameters on jinja templates by using the ``{{context.params}}`` dictionary.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in ``globals()``. Which means you need to make sure to have

Review comment:
       (possibly need 4 spaces for the note block, not sure.)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710489888


   [The Workflow run](https://github.com/apache/airflow/actions/runs/311319137) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r511147200



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parameterize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+You can also use the parameters on jinja templates by using the ``{{context.params}}`` dictionary.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in ``globals()``. Which means you need to make sure to have

Review comment:
       Looks good, downloaded artifact from the build-docs job 
   ![image](https://user-images.githubusercontent.com/8811558/97052477-fc6a0f00-1578-11eb-8e0b-8c7280aea321.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.

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



[GitHub] [airflow] casassg commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r511031290



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Added to updating.md as well.

##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Yup. let me know if that's okay

##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parameterize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+You can also use the parameters on jinja templates by using the ``{{context.params}}`` dictionary.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in ``globals()``. Which means you need to make sure to have

Review comment:
       I dont think so, if you look above, other notes have it inlined. See: https://raw.githubusercontent.com/apache/airflow/3d99f29fbf5da3e28bb26feb12d17ef5928b147e/docs/concepts.rst




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510924557



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       Right now it would have to be `DAG = your_dag_function()` -- and you also need the string `airflow` in the dag files somewhere `return all(s in content for s in (b'DAG', b'airflow'))` from  https://github.com/apache/airflow/blob/5fc06a6ef9c136a4861bc2426aea768e9a18dcbb/airflow/utils/file.py#L192-L211




----------------------------------------------------------------
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.

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



[GitHub] [airflow] casassg commented on pull request #10587: [WIP] Start @dag decorator

Posted by GitBox <gi...@apache.org>.
casassg commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-687268164


   That's totally okay w me, mostly not having a lot of bandwidth to work on this lately, so please go ahead :D


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510945795



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,31 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.
+DAG decorator creates a DAG generator function. This function when called returns a DAG.
+
+DAG decorator also sets up the parameters you have in the function as DAG params. This allows you to parametrize
+your DAGs and set the parameters when triggering the DAG manually. See
+:ref:`Passing Parameters when triggering dags <dagrun:parameters>` to learn how to pass parameters when triggering DAGs.
+
+Example DAG with decorator:
+
+.. exampleinclude:: /../airflow/example_dags/example_dag_decorator.py
+    :language: python
+    :dedent: 4
+    :start-after: [START dag_decorator_usage]
+    :end-before: [END dag_decorator_usage]
+
+.. note:: Note that Airflow will only load DAGs that appear in``globals()``. Which means you need to make sure to have
+  a variable for your returned DAG in the module scope. Otherwise Airflow won't detect your decorated DAG. In addition,
+  you may want to make your dag variable named ``DAG`` such that Airflow doesn't skip this module when the
+  ``DAG_DISCOVERY_SAFE_MODE`` is activated.

Review comment:
       @casassg Has updated that check: https://github.com/apache/airflow/pull/10587/files#diff-d089cab1f4d8262ade0db0e327e7d094d2e8ecd592db597571a5c0a4a3d35ef8R213-R214




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #10587: @dag decorator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506207983



##########
File path: airflow/models/dagparam.py
##########
@@ -0,0 +1,85 @@
+# 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.
+
+from typing import Any, Dict, Callable
+
+from inspect import signature
+
+
+import functools
+
+
+class DagParam:
+    """
+    Class that represents a DAG run parameter.
+
+    It can be used to parametrized your dags.
+
+    **Example**:
+
+        with DAG(...) as dag:
+          EmailOperator(subject=dag.param('subject', 'Hi from Airflow!'))
+
+    This object can be used in legacy Operators via Jinja.
+
+    :param current_dag: Dag that will be used to pull the parameter from.
+    :type current_dag: airflow.models.dag.DAG
+    :param name: key value which is used to set the parameter
+    :type name: str
+    :param default: Default value used if no parameter was set.
+    :type default: Any
+    """
+
+    def __init__(self, current_dag, name: str, default: Any):
+        current_dag.params[name] = default
+        self._name = name
+        self._default = default
+
+    def resolve(self, context: Dict) -> Any:
+        """
+        Pull DagParam value from DagRun context. This method is run during ``op.execute()``
+        in respectable context.
+        """
+        return context.get('params', {}).get(self._name, self._default)
+
+
+def dag(*dag_args, **dag_kwargs):

Review comment:
       Should this maybe live in airflow.models.dag?
   
   Additionally this should be imported/re-exported from airflow.decorators.
   

##########
File path: tests/models/test_dagparam.py
##########
@@ -0,0 +1,80 @@
+import unittest
+from datetime import datetime, timedelta
+
+from airflow.jobs.scheduler_job import TI
+from airflow.models import DagRun
+from airflow.models.dag import DAG
+from airflow.models.dagparam import dag as dag_decorator
+from airflow.operators.python import (
+    PythonOperator,
+    task)
+from airflow.utils import timezone
+from airflow.utils.state import State
+from airflow.utils.types import DagRunType
+from tests.test_utils.config import conf_vars
+from airflow.utils.session import create_session
+from airflow.utils.state import State
+from airflow.utils.types import DagRunType
+from tests.test_utils.db import clear_db_runs
+
+DEFAULT_ARGS = {
+    "owner": "test",
+    "depends_on_past": True,
+    "start_date": timezone.utcnow(),
+    "retries": 1,
+    "retry_delay": timedelta(minutes=1),
+}
+DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+
+VALUE = 42
+
+
+class TestDagParamRuntime(unittest.TestCase):
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_xcom_pass_to_op(self):
+        with DAG(dag_id="test_xcom_pass_to_op", default_args=DEFAULT_ARGS) as dag:
+            value = dag.param('value', default=VALUE)
+            @task
+            def return_num(num):
+                return num
+
+            xcom_arg = return_num(value)
+
+        dr = dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        xcom_arg.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull() == VALUE
+
+
+class TestDagDecorator:
+    @conf_vars({("core", "executor"): "DebugExecutor"})
+    def test_xcom_pass_to_op(self):
+
+        @dag_decorator(default_args=DEFAULT_ARGS)
+        def test_pipeline(some_param, other_param=VALUE):

Review comment:
       Am I reading this right the the `@dag` decorator essentially makes a function in to a DAG generator?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710670236


   [The Workflow run](https://github.com/apache/airflow/actions/runs/311510535) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r510918571



##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.

Review comment:
       Can you add a cross-reference to the context manger section

##########
File path: docs/concepts.rst
##########
@@ -162,6 +162,32 @@ Example DAG with decorated style
         html_content=email_info['body']
     )
 
+DAG decorator
+-------------
+
+.. versionadded:: 2.0.0
+
+In addition to creating DAGs using context managed, in Airflow 2.0 you can also create DAGs from a function.

Review comment:
       Can you add a cross-reference to the context manager section




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506655564



##########
File path: tests/models/test_dag.py
##########
@@ -1754,3 +1757,98 @@ def test_count_number_queries(self, tasks_count):
                 state=State.RUNNING,
                 execution_date=TEST_DATE,
             )
+
+
+class TestDagDecorator:
+    DEFAULT_ARGS = {
+        "owner": "test",
+        "depends_on_past": True,
+        "start_date": timezone.utcnow(),
+        "retries": 1,
+        "retry_delay": timedelta(minutes=1),
+    }
+    DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+    VALUE = 42
+
+    def setUp(self):
+        super().setUp()
+        self.operator = None
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_set_dag_id(self):
+        @dag_decorator('test', default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'test'

Review comment:
       Should we add test that asserts that the returned value is a `DAG`? This will increase understanding imho 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#discussion_r506656365



##########
File path: tests/models/test_dag.py
##########
@@ -1754,3 +1757,98 @@ def test_count_number_queries(self, tasks_count):
                 state=State.RUNNING,
                 execution_date=TEST_DATE,
             )
+
+
+class TestDagDecorator:
+    DEFAULT_ARGS = {
+        "owner": "test",
+        "depends_on_past": True,
+        "start_date": timezone.utcnow(),
+        "retries": 1,
+        "retry_delay": timedelta(minutes=1),
+    }
+    DEFAULT_DATE = timezone.datetime(2016, 1, 1)
+    VALUE = 42
+
+    def setUp(self):
+        super().setUp()
+        self.operator = None
+
+    def tearDown(self):
+        super().tearDown()
+        clear_db_runs()
+
+    def test_set_dag_id(self):
+        @dag_decorator('test', default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'test'
+
+    def test_arg_not_set_fail(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def noop_pipeline(value):
+            @task
+            def return_num(num):
+                return num
+
+            return_num(value)
+        with pytest.raises(TypeError):
+            noop_pipeline()
+
+    def test_dag_id_function_name(self):
+        @dag_decorator(default_args=self.DEFAULT_ARGS)
+        def noop_pipeline():
+            @task
+            def return_num(num):
+                return num
+
+            return_num(4)
+
+        assert noop_pipeline().dag_id, 'noop_pipeline'

Review comment:
       This test is exactly same as `test_set_dag_id` if I'm not mistaken 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on pull request #10587: Add @dag decorator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #10587:
URL: https://github.com/apache/airflow/pull/10587#issuecomment-710332133


   Definitely adding example DAG will help with testing. Apart from that, how does it play with TaskGroups, do you think there can be some problems? 


----------------------------------------------------------------
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.

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