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/05/21 21:45:43 UTC

[GitHub] [airflow] casassg opened a new pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Airflow [AIP-31](https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-31:+Airflow+functional+DAG+definition) task decorator implementation. This decorator should facilitate wrapping a function into an operator and use it as such in a DAG. Closes #8057 + #8056.
   
   - Should be used without args or with args/kwargs for the underlying operator:
   ```
   @task
   def simple_task(...):
   
   @operator(dag=dag)
   def simple_task(...):
   ```
   - Task ID should be the function name by default.
   - Decorator should return an instance of `PythonFunctionalOperator`. This can be used to set task dependencies. Ex:
   ```
   @task 
   def simple_task(...)
     pass
   
   simple_task >> another_task
   ```
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [ X ] Description above provides context of the change
   - [ ] Unit tests coverage for changes (not needed for documentation changes)
   - [ ] Target Github ISSUE in description if exists
   - [ ] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [ ] Relevant documentation is updated including usage instructions.
   - [ ] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more 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.

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



[GitHub] [airflow] casassg commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   I prefer to avoid doing that. Seems to add quite a bit of code complexity and I doubt it will impact significantly the user experience. I think it should be fine from users to use `airflow.decorators.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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,145 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = PythonOperator. ui_color

Review comment:
       ```suggestion
       ui_color = PythonOperator.ui_color
   ```




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +214,60 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+*Added in Airflow 1.10.11*
+
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+If you call a decorated function twice in a DAG, it will error as it doesn't know what args to use.
+If you want to reuse decorated functions, use the copy method as follows:

Review comment:
       updated.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):

Review comment:
       Why from BaseOperator, and not from PythonOperator?




----------------------------------------------------------------
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] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       You are using F-Strings here, too - not only in examples.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       @dimberman you can name them manually as well. `@task(task_id='my_task')` Added a test for 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.

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



[GitHub] [airflow] ashb commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/models/xcom_arg.py
##########
@@ -83,7 +83,7 @@ def __getitem__(self, item):
         """
         Implements xcomresult['some_result_key']
         """
-        return XComArg(operator=self.operator, key=item)
+        return XComArg(operator=self.operator, key=str(item))

Review comment:
       Any chance you could pull this (and the test for it) out to a separate PR please?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       Same tbh. I think given the discussion on calling a decorated function several times, I'll refactor to accomodate that. This should simplify code a bit 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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       I see another issue here. Currently, if `task_id` is not provided user will get `KeyError: 'task_id'` instead of `TypeError: __init__() missing 1 required positional argument: 'task_id' `
   
   Also, this seems to work as expected:
   ```python
   In [8]: class CustomOp(BaseOperator):
      ...:     def __init__(self, a, b, *args, **kwargs):
      ...:         super().__init__(*args, **kwargs)
      ...:         self.task_id = "other task id"
      ...:
   
   In [9]: op = CustomOp(a=1, b=2, task_id="task_id")
   
   In [10]: op.task_id
   Out[10]: 'other task id'
   ```
   




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       Also, not sure if we should use the same approach from parametrized, as you may want to run an operation twice with the same args/kwargs. Also, if the arg/kwarg is a XComArg, what should we do?




----------------------------------------------------------------
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] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs

Review comment:
       Yes, I think that would be better. +1, for dropping it.

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       So if we want this in 1.10.11 we need to be very careful :/

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Agree with turbaszek. This is not self-explanatory at all. I think having `update_user(user_id, task_id=...)` would be much better. We can access the function signature inside `task`. This should work and is more self-explanatory.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       To me the "unrolling" functionality seems to create overhead here. Why can't you push the objects pickled or as json string to xcom? If you want to have it unrolled this (in my opinion) should be done by the user.

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       Unfortunately we can't use f-Strings, because we still support Python 2.7 in the 1.10.x series.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)

Review comment:
       created #9041 as a follow up 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.

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



[GitHub] [airflow] turbaszek commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   @casassg thanks for your work! πŸš€ 


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Rebased from latest master to see if integration tests are fixed.


----------------------------------------------------------------
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] boring-cyborg[bot] commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #8962:
URL: https://github.com/apache/airflow/pull/8962#issuecomment-632362835


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, pylint and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/master/docs/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/master/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better πŸš€.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://apache-airflow-slack.herokuapp.com/
   


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Hi @casassg is there any way we can help you with moving on? πŸ˜‰ 


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Sent update vote to mailing list - https://lists.apache.org/thread.html/rc40ea8171be80365c46ce29b4c8cfacc7567963c0df6af6056537c63%40%3Cdev.airflow.apache.org%3E - will ask for merge on Thursday if there's no opposition to the change (lazy consensus) as proposed by Ash.


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.

Review comment:
       "Airflow will also automatically add dependencies between tasks to ensure that XCom messages are available when operators are executed."
   
   wdyt?

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)

Review comment:
       I rather do a more complete E2E tutorial here. We may want to switch the Hello World example on the tutorial page with an even simpler example. Thoughts?

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'

Review comment:
       mostly, except:
   ```py
   assert do_run.task_id == 'do_run'
   ```
   
   `run` here is an XComArg representing the value returned in the do_run operator.  




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Do I correctly understand that this will not work?
   ``` python
   @task 
   def update_user(user_id: str):
       ...
   
   with DAG(...):
       # Fetch list of users 
       ...
       # Execute task for each user
       for user_id in users_list:
           update_user(user_id)
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.

Review comment:
       ```suggestion
           """
           Python wrapper to generate PythonFunctionalOperator out of simple python functions.
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       Personally I would prefer to use typehints to indicate multiple output than a flag. It will solve the issue and add more information to task definitions. Of course, typehints are optional but we can require them to make multiple outputs work. Here's a similar thing from PySpark:
   
   https://databricks.com/blog/2020/05/20/new-pandas-udfs-and-python-type-hints-in-the-upcoming-release-of-apache-spark-3-0.html

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):

Review comment:
       ```suggestion
       def _get_unique_task_id(task_id: str, dag: DAG) -> str:
   ```

##########
File path: airflow/ti_deps/deps/trigger_rule_dep.py
##########
@@ -18,10 +18,10 @@
 
 from collections import Counter
 
-import airflow
 from airflow.ti_deps.deps.base_ti_dep import BaseTIDep
 from airflow.utils.session import provide_session
 from airflow.utils.state import State
+from airflow.utils.trigger_rule import TriggerRule as TR

Review comment:
       Is this a related change? 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       That's true. Mostly saying as a full refactor of user docs on how to write DAGs may be more adequate for Airflow 2.0 than 1.10. But happy to reconsider this. If we go down that path, we should do it in a separate PR though. This PR is already become large enough




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       List/tuples is just an easy way to send several outputs w no specific name. Changing ordering should be transparent, also tuples and lists are ordered by default so it should be fine?
   
   Default is off to avoid doing calculation unless explicit. Can change it to on if needed.
   
   From the other frameworks only databand supports it I believe. Dagster has a different mechanism and Prefect has an indexing mechanism that may work w numbers? not sure https://docs.prefect.io/core/concepts/tasks.html#indexing
   
   CC @evgenyshulman 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       And we can discuss this _after_ this PR is merged if we want. It's not a blocker on merging the PR.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       `_PythonFunctionalOperator` is a private operator. Aka it should only be used with `@task` which does always set the task_id. 
   
   Will make the field mandatory just in case. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       I'm still not sure why do we want to make task signature more generic. Your proposal basically would mean that I can do: `@task('test')` I still need to check that the first arg (if exists) is a callable. I don't see much difference in declaring the wrapper function before or after the check.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       `_PythonFunctionalOperator` is a private operator. Aka it should only be used with `@task` which does always set the task_id. 
   
   Will make the field mandatory just in case. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       I'm still not sure why do we want to make task signature more generic. Your proposal basically would mean that I can do: `@task('test')` I still need to check that the first arg (if exists) is a callable. I don't see much difference in declaring the wrapper function before or after the check.




----------------------------------------------------------------
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 edited a comment on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #8962:
URL: https://github.com/apache/airflow/pull/8962#issuecomment-632646688


   > The main question is: do we want to support that? Is this "hack" worth it, and do IDEs get massively confused by this?
   
   I think if IDEs support it, and we add tests for it I'd say, yes.
   
   I wonder if we can do this in `airflow/__init__.py` (adding the task decorator import)
   
   ```python
   STATICA_HACK = True
   globals()['kcah_acitats'[::-1].upper()] = False
   if STATICA_HACK:  # pragma: no cover
       from airflow.models.dag import DAG
       from airflow.exceptions import AirflowException
       from airflow.decorators import task
   ```
   
   Without confusing IDEs more.


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       I see another issue here. Currently, if `task_id` is not provided user will get `KeyError: 'task_id'` instead of TypeError: __init__() missing 1 required positional argument: 'task_id' 
   




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Main worry is that then what is `update_user`. What you are describing here is using `update_user` as an operator factory. It has it's value, but it also feels too magic to me atm. If `update_user` is a factory, then you can't change the operator instance at all or use it to set non-data dependencies. 
   
   We could capture task_id kwarg and generate a new operator, but then what is `update_user` the first operator, the latest one? What does `update_user` represent?
   
   You can either do (1) `update_user(i) for i in range(20)` or (2) `update_user >> other_operation`, but not both. I prefer to support 2nd option as it adapts more to what Airflow already does with operators.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Sent message to mailing list: https://lists.apache.org/thread.html/rc40ea8171be80365c46ce29b4c8cfacc7567963c0df6af6056537c63%40%3Cdev.airflow.apache.org%3E




----------------------------------------------------------------
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 pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   > The main question is: do we want to support that? Is this "hack" worth it, and do IDEs get massively confused by this?
   
   I think if IDEs support it, and we add tests for it I'd say, yes.


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,144 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        _old_kwargs = self._kwargs
+        if task_id:
+            _old_kwargs['task_id'] = task_id

Review comment:
       It totally will break dag serialization.




----------------------------------------------------------------
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] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python function to an Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code-block:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+      @dag.task
+      def hello_world():
+          print('hello world!')
+
+
+      # Also...
+      from airflow.decorators import task
+
+
+      @task
+      def hello_name(name: str):
+          print(f'hello {name}!')
+
+
+      hello_name('Airflow users')
+
+Task decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function more than once in a DAG. The decorated function will automatically generate
+a unique ``task_id`` for each generated operator.
+
+.. code-block:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def update_user(user_id: int):
+      ...
+
+    # Avoid generating this list dynamically to keep DAG topology stable between DAG runs
+    for user_id in user_ids:
+      update_current(user_id)

Review comment:
       ```suggestion
         update_user(user_id)
   ```




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):

Review comment:
       Mainly this came from a realization that we were reimplementing PythonOperator either way. Main discussion here: https://github.com/databand-ai/airflow/pull/5#discussion_r416168956

##########
File path: airflow/models/xcom_arg.py
##########
@@ -83,7 +83,7 @@ def __getitem__(self, item):
         """
         Implements xcomresult['some_result_key']
         """
-        return XComArg(operator=self.operator, key=item)
+        return XComArg(operator=self.operator, key=str(item))

Review comment:
       related discussion: https://github.com/databand-ai/airflow/pull/5#discussion_r416171596 
   
   This small cast makes the `multiple_output` piece work more reliably and more transparently to the user. Not actually a new feature, but mostly a fix of what already got merged in XComArg.
   
   I can break it into a separate PR but note that XCom class already does this when saving the key (not when retrieving it)

##########
File path: airflow/ti_deps/deps/trigger_rule_dep.py
##########
@@ -18,10 +18,10 @@
 
 from collections import Counter
 
-import airflow
 from airflow.ti_deps.deps.base_ti_dep import BaseTIDep
 from airflow.utils.session import provide_session
 from airflow.utils.state import State
+from airflow.utils.trigger_rule import TriggerRule as TR

Review comment:
       Had to resolve it because it created a circular import when I added the decorator as `airflow.task`

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       It will fail at parse time. 
   
   Correct. 1 function == 1 operator. Airflow doesn't allow dynamic operators (execute 1 operator several times). You can still work around it. 
   
   This will work though:
   
   ```py
   @task 
   def update_user(user_id: str):
       ...
   
   with DAG(...):
       # Fetch list of users 
       ...
       # Execute task for each user
       for user_id in users_list:
           update_user.copy(f'update_{user_id}')(user_id)

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       The AIP used `__call__` instead which was a bit more verbose and cofusing in my opinion. I rather keep `.output` as it allows for more composable DAGs.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):

Review comment:
       Main reason for this is to allow for `copy` method to work without having to specify explicitly a new task_id. This was mostly for convenience to the user. While saying this aloud, I think maybe we can move the generate portion to `copy` method better. 
   
   related conversation: https://github.com/databand-ai/airflow/pull/5#discussion_r417771808

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       Defining a task decorated operator without a DAG and adding it to the DAG on `__call__`. 
   
   ```
   @taks 
   def add_2(num)
     return num+2
   
   with DAG(...):
      add_2(2)
   ```
   Otherwise this does not work. Also if we define `default_args` in `DAG` we wont be able to capture it either. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       I like the typehints, but would probably leave it for a follow up PR. Rather make it explicit. 
   
   Happy to revert unrolling tuples and lists.

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43
+            }
+
+        test_number = 10
+        with self.dag:
+            return_dict(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_dict.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='number') == test_number + 1
+        assert ti.xcom_pull(key='43') == 43
+        assert ti.xcom_pull() == {'number': test_number + 1, '43': 43}
+
+    def test_tuple_outputs(self):
+        """Tests pushing multiple outputs as tuple"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return number + 1, 43
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_list_outputs(self):
+        """Tests pushing multiple outputs as list"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return [number + 1, 43]
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_xcom_arg(self):
+        """Tests that returned key in XComArg is returned correctly"""
+
+        @task_decorator
+        def add_2(number: int):
+            return number + 2
+
+        @task_decorator
+        def add_num(number: int, num2: int = 2):

Review comment:
       They do. I can add a test.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):

Review comment:
       It does otherwise you can't do:
   ```
   @task(task_id='test', dag=dag, multiple_outputs=True)
   def random_name(...)
   
   ``` 
   This calls function and then wraps it. We need to allow both use cases (using args in decorator and not using it).

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43

Review comment:
       XCom automatically casts keys to strings. That's the reason for adding casting in XComArg/




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*

Review comment:
       ```suggestion
   ```
   
   Let's not put this in until we're _sure_ it's going to make it in to the release. Sorry.
   
   I would _love_ to have this in 1.10.11, but it might not make it in. We can add this in a separate PR once we've know for sure if it makes it in or not




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Random thought. Functional dags is perhaps not the right name for this whole feature. It's not like functional programming where a something is run without side-effects.
   
   What would people think about renaming this to "Literate DAGs"? I'll take to the mailing list if people agree, just something doesn't quite sit right with me about "functional" today.
   
   If no one else likes this/thinks functional is great then I'll not worry.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Hm, I have mixed feelings:
   - `copy` is not self-explanatory in this case imho
   - in such case, shouldn't we generate auto id? Or at least can we try to do `update_user(user_id, task_id=user_id)`




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       `_PythonFunctionalOperator` is a private operator. Aka it should only be used with `@task` which does always set the task_id. 
   
   Will make the field mandatory just in case. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       I'm still not sure why do we want to make task signature more generic. Your proposal basically would mean that I can do: `@task('test')` I still need to check that the first arg (if exists) is a callable. I don't see much difference in declaring the wrapper function before or after the check.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):

Review comment:
       I'm not 100% sure what's going on here, but I'm not sure I like the idea of autogenerating task_ids.
   
   I think either the task_id should be the function name, or let the user specify it via an argument to the `@task` decorator

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       What use case/code path is this enabling?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):

Review comment:
       Which the decorator already does, so I don't follow what this is for. Can you explain please? (And perhaps add it as a comment to the source)

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id

Review comment:
       Why is copy modifying the source object? That doesn't seem like it should.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Is this a exeuction time error, or does this happen at parse time?
   
   We don't appear to have any tests that cover this (or I missed it). Can you add some, and add a section about this to the docs.

##########
File path: airflow/ti_deps/deps/trigger_rule_dep.py
##########
@@ -18,10 +18,10 @@
 
 from collections import Counter
 
-import airflow
 from airflow.ti_deps.deps.base_ti_dep import BaseTIDep
 from airflow.utils.session import provide_session
 from airflow.utils.state import State
+from airflow.utils.trigger_rule import TriggerRule as TR

Review comment:
       This change seems unrelated?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       I think I'd be +1 for only supporting "unrolling" of dicts -- unrolling lists/tuples makes the consuming code harder to follow:
   
   ```python
   @dag.task
   def x():
     return "a", "b"
   
   r = x()
   email(r[1])  # What is r[1]
   ```
   
   We _could_ support it, but I feel encouraging users to write better code might be a good idea? Don't know.

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Do we still need this `.output`? The AIP didn't have that, and its much "nicer" without it.

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)

Review comment:
       For the basic example I think it might be clearer if we didn't use "multiple_outputs"

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.

Review comment:
       "from the message dependencies" isn't quite clear - what it means. (What's a message? _I_ know what you mean, but we need to reword this)

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):

Review comment:
       ```suggestion
   class TestAirflowTaskDecorator(unittest.TestCase):
   ```

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43
+            }
+
+        test_number = 10
+        with self.dag:
+            return_dict(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_dict.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='number') == test_number + 1
+        assert ti.xcom_pull(key='43') == 43
+        assert ti.xcom_pull() == {'number': test_number + 1, '43': 43}
+
+    def test_tuple_outputs(self):
+        """Tests pushing multiple outputs as tuple"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return number + 1, 43
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_list_outputs(self):
+        """Tests pushing multiple outputs as list"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return [number + 1, 43]
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_xcom_arg(self):
+        """Tests that returned key in XComArg is returned correctly"""
+
+        @task_decorator
+        def add_2(number: int):
+            return number + 2
+
+        @task_decorator
+        def add_num(number: int, num2: int = 2):
+            return number + num2
+
+        test_number = 10
+
+        with self.dag:
+            bigger_number = add_2(test_number)
+            ret = add_num(bigger_number, XComArg(add_2))
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        add_2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+        add_num.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+        ti_add_num = [ti for ti in dr.get_task_instances() if ti.task_id == 'add_num'][0]
+        assert ti_add_num.xcom_pull(key=ret.key) == (test_number + 2) * 2  # pylint: disable=maybe-no-member
+
+    def test_dag_task(self):
+        """Tests dag.task property to generate task"""
+
+        @self.dag.task
+        def add_2(number: int):
+            return number + 2
+
+        test_number = 10
+        ret = add_2(test_number)
+
+        dr = self.dag.create_dagrun(

Review comment:
       At this point, it seems like we are duplicating a lot of the behavior in previous tests.
   
   Rather than running the DAG again here, it might be better to more directly test dag.tasks etc, and not run it.

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43
+            }
+
+        test_number = 10
+        with self.dag:
+            return_dict(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_dict.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='number') == test_number + 1
+        assert ti.xcom_pull(key='43') == 43
+        assert ti.xcom_pull() == {'number': test_number + 1, '43': 43}
+
+    def test_tuple_outputs(self):
+        """Tests pushing multiple outputs as tuple"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return number + 1, 43
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_list_outputs(self):
+        """Tests pushing multiple outputs as list"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return [number + 1, 43]
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_xcom_arg(self):
+        """Tests that returned key in XComArg is returned correctly"""
+
+        @task_decorator
+        def add_2(number: int):
+            return number + 2
+
+        @task_decorator
+        def add_num(number: int, num2: int = 2):
+            return number + num2
+
+        test_number = 10
+
+        with self.dag:
+            bigger_number = add_2(test_number)
+            ret = add_num(bigger_number, XComArg(add_2))
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        add_2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+        add_num.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+        ti_add_num = [ti for ti in dr.get_task_instances() if ti.task_id == 'add_num'][0]
+        assert ti_add_num.xcom_pull(key=ret.key) == (test_number + 2) * 2  # pylint: disable=maybe-no-member
+
+    def test_dag_task(self):
+        """Tests dag.task property to generate task"""
+
+        @self.dag.task
+        def add_2(number: int):
+            return number + 2
+
+        test_number = 10
+        ret = add_2(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        add_2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key=ret.key) == test_number + 2  # pylint: disable=maybe-no-member
+
+    def test_dag_task_multiple_outputs(self):
+        """Tests dag.task property to generate task with multiple outputs"""
+
+        @self.dag.task(multiple_outputs=True)
+        def add_2(number: int):
+            return number + 2, 42
+
+        test_number = 10
+        ret = add_2(test_number)
+
+        dr = self.dag.create_dagrun(

Review comment:
       Likewise here, this is _almost_ duplicating `test_list_outputs`

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)

Review comment:
       Rather than having to special case it like this, can we use https://docs.python.org/3/library/unittest.mock.html#any

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):

Review comment:
       ```suggestion
   def task(python_callable: Callable, **kwargs):
   ```
   
   It doesn't make sense to have the callable be optional.

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'

Review comment:
       ```suggestion
               run = do_run()
              
               do_run_1 = do_run.copy()
               do_run_2 = do_run.copy()
               assert ['do_run', 'do_run__1', 'do_run__2'] == self.dag.task_ids
               assert run.task_id == 'do_run'
               
           assert do_run_1.task_id == 'do_run__1'
           assert do_run_2.task_id == 'do_run__2'
   ```
   
   (I think)

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43

Review comment:
       We should test (and work out what we want) when you do `return { 43: 43, '43': 42 }`

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43
+            }
+
+        test_number = 10
+        with self.dag:
+            return_dict(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_dict.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='number') == test_number + 1
+        assert ti.xcom_pull(key='43') == 43
+        assert ti.xcom_pull() == {'number': test_number + 1, '43': 43}
+
+    def test_tuple_outputs(self):
+        """Tests pushing multiple outputs as tuple"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return number + 1, 43
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_list_outputs(self):
+        """Tests pushing multiple outputs as list"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_tuple(number: int):
+            return [number + 1, 43]
+
+        test_number = 10
+        with self.dag:
+            return_tuple(test_number)
+
+        dr = self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            start_date=timezone.utcnow(),
+            execution_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        return_tuple.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ti = dr.get_task_instances()[0]
+        assert ti.xcom_pull(key='0') == test_number + 1
+        assert ti.xcom_pull(key='1') == 43
+        assert ti.xcom_pull() == [test_number + 1, 43]
+
+    def test_xcom_arg(self):
+        """Tests that returned key in XComArg is returned correctly"""
+
+        @task_decorator
+        def add_2(number: int):
+            return number + 2
+
+        @task_decorator
+        def add_num(number: int, num2: int = 2):

Review comment:
       Do defaults work? We should add a test for that I think.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')

Review comment:
       ```suggestion
               raise TypeError('`python_callable` param must be callable')
   ```
   
   Python has a built in exception class designed for this exact purpose, lets use it.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       ```suggestion
       _PythonFunctionalOperator._validate_python_callable(python_callable)
       return wrapper(python_callable)
   ```
   
   I think is what we should do -- let's validate this as _early_ as possible.
   
   What use case/pattern does allowing it to be not specified give us?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,61 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function twice in a DAG. The decorated function will automatically generate unique a ``task_id``
+for each generated operator.
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def update_user(user_id: int):
+      ...
+
+    for user_id in user_ids:

Review comment:
       Added comment.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,61 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function twice in a DAG. The decorated function will automatically generate unique a ``task_id``
+for each generated operator.
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def update_user(user_id: int):
+      ...
+
+    for user_id in user_ids:

Review comment:
       Adding a note `Avoid generating this list dynamically to keep dag topology stable between DAG runs`
   
   If we have dag serialization and dag versioning, that should be fine in the future though right? πŸ‘€ 
   
   

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       That's fair. The Functional DAGs comes from the AIP title itself. I do understand the confusion, but not sure if literate is better. 
   
   Ideally though, having XComArg and all this, it should allow DAGs to be written without as much side effects. You will be able to paramatrize an operator such that it is indeed more functional (takes A and B, outputs C) instead of current approach where you mostly generate artifacts that depend on Airflow context and such they are side effects.
   
   Not sure if the above makes sense. In my mind, it makes it more functional (less prone to side effects) as you make operators resemble more functions, and XComArgs map to function parameters and function outputs. 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Should we make an issue for that, discuss and then adjust the docs in a follow-up PR?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Created #9415 




----------------------------------------------------------------
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 pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Can you please rebase the PR on master


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Could you update the AIP, and draft a short summary change to the list with lazy consensus (i.e. if no one objects within 48 hours we'll merge it) please?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       No strong opinion here, I just think this is more explicit 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')

Review comment:
       What about class methods?




----------------------------------------------------------------
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] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       There are more cases where you are using F-Strings https://github.com/apache/airflow/pull/8962#discussion_r431670798




----------------------------------------------------------------
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] evgenyshulman commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   @casassg  great initiative and awesome implementation! 


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43

Review comment:
       Ooh I see, sorry did not fully understand the issue. Not sure what the best path for this may be. We can raise an exception (not great) or log weird usage. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs

Review comment:
       Not sure what you mean here. How then is the operator supposed to pass down kwargs like `owner` that is a BaseOperator valid kwarg? If I remove kwargs here, then I won't be able to set owner in this operator.

##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       Given this is an example DAG on a document, I don't think it really matters that much. If users want to use it on python 2 (which btw is already deprecated) they can change it to format. But I think we should adapt to use py3 examples better.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,144 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        _old_kwargs = self._kwargs
+        if task_id:
+            _old_kwargs['task_id'] = task_id

Review comment:
       Updated, this should not change `self._kwargs` and it works (check tests).

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)

Review comment:
       thought Airflow was using pytest for testing, can't seem to find an equivalent for ANY there. Btw, this is copied from PythonTest. Mostly copied the exact same code as proposed by @turbaszek to avoid weird side effects when inheriting TestCases.




----------------------------------------------------------------
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] dimberman commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       Awesome thanks @casassg 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       we also use type annotations which is py3 only. We should probs decide if we want to support Python2 or not at all. 




----------------------------------------------------------------
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] potiuk commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Just in time for the Summit !


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XCom values <concepts:xcom>`. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Airflow will also automatically add dependencies between
+tasks to ensure that XCom messages are available when operators are executed.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: str) -> str:

Review comment:
       This type hint is wrong.
   
   ```suggestion
       def prepare_email(raw_json: str) -> dict:
   ```
   
   Or `Dict[str,str]` if you want to be more precise.




----------------------------------------------------------------
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] aoen commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()

Review comment:
       Will this function play nicely if neither side of the or returns true and task_id is None, e.g. if someone initializes a task and then adds it to a DAG later?
   
   I guess it doesn't really make sense with this pattern so specify dag_id later, so I think raising an exception if DAG is not specified would be reasonable (+accompanying unit test).

##########
File path: airflow/models/xcom_arg.py
##########
@@ -83,7 +83,7 @@ def __getitem__(self, item):
         """
         Implements xcomresult['some_result_key']
         """
-        return XComArg(operator=self.operator, key=item)
+        return XComArg(operator=self.operator, key=str(item))

Review comment:
       Why the cast? Seems being explicit about the type received here might be safer, and the cast can happen on the caller side, otherwise this weakens type safety of this arg for all XCOM use cases.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       I can definitely see the value in unrolling dictionaries, but I'm curious for lists/tuples, especially since it seems a bit brittle (changing ordering of elements/removing elements/etc will break consuming code).
   
   Curious why default is off, seems like defaulting it on could be nice for the prototyping/new user use-cases.
   
   My concern here is we are exposing an interface we will need to support that does not provide significant value to users that will increase complexity (e.g. the `  if not kwargs.get('do_xcom_push', True) and not multiple_outputs:` line. I know you've being working with some other Pipeline execution abstractions, do they support similar things?
   
   To other committers/reviewers: I think this should warrant special attention in this review since this is going to become part of the public API and hard to change in the future.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()

Review comment:
       Why default 'dag'/task_id to None?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)

Review comment:
       Nit: seems like you can make this a more generic validation method and stick the if not callable... line in there too.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(

Review comment:
       Nit: prefer list comprehension
   ` [task_id for task_id in dag.task_ids if task_id.startswith(prefix)`

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1

Review comment:
       What if there is more than 9 tasks? I think the [-1] breaks. Would recommend regex approach mentioned previously to be safe + some more unit tests. Maybe look to see if a late-binding approach is possible since that's cleanest but I'm not sure...

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1

Review comment:
       nit: s/num/suffix

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),

Review comment:
       Seems like this section could use better test coverage to test different task_ids

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:

Review comment:
       Rather than error out here does it make sense to automatically set a default here?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]

Review comment:
       Would a safer approach be using a regex for 
   
   > __digit 
   
   so
   ddavydov__ or ddavydov__addition doesn't break this? Best might be lazily binding these tasks, only generating the indices once the DAG is "finalized" to make things both safe and simple (don't need to parse the task ids), but that might be too complicated since I'm not sure where exactly the DAG would be finalized.
   
   __ is reused so should probably be a constant.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Curious why the functions can't be reused seems a bit annoying for users, wonder if we can fix this (e.g. late-binding the task_ids or something...) or add a TODO, might be worth adding a comment here. ._checking if it was already called feels a bit hacky as it kind of couples the task execution with global state.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]

Review comment:
       Nit: might call this task_id_root or something instead of prefix since it's not really a prefix (since words have prefixes/suffixes, and the core is called a root).

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:

Review comment:
       Nit: Consider moving this task id generation into it's own function (easier to test/makes this function a bit more lean).

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs

Review comment:
       How does the use of kwargs relate to their deprecation in Airflow 2.0:
   e.g. in the BaseOperator code:
   ```
   
                 if args or kwargs:
               # TODO remove *args and **kwargs in Airflow 2.0
               warnings.warn(
                   'Invalid arguments were passed to {c} (task_id: {t}). '
                   'Support for passing such arguments will be dropped in '
                   'Airflow 2.0. Invalid arguments were:'
                   '\n*args: {a}\n**kwargs: {k}'.format(
                       c=self.__class__.__name__, a=args, k=kwargs, t=task_id),
                   category=PendingDeprecationWarning,
                   stacklevel=3
               )
   ```
   
   
   Does this need a deprecation warning too? Should we just not allow kwargs/args here in the first place, or is it needed for backwards compatibility?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '

Review comment:
       Is there enough context printed here with the offending task id/dag id (and same for other exceptions in both this function and others)?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       Not great that this and potential parse-time errors that could occur here is moved to runtime instead of DAG parse time, another reason it might be worth thinking about a more parse-time friendly solution if possible (or what's stopping Airflow from supporting this at the current time).




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       I've created a issue for future https://github.com/apache/airflow/issues/8996




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       > We could capture task_id kwarg and generate a new operator, but then what is `update_user` the first operator, the latest one? What does `update_user` represent?
   
   For me `update_user` is a function and as a function in can be called many times with different input thus yielding different results. I have never meet "function as a singleton" pattern. If we don't want to generate `task_id` for users then we may consider raising an exception on second invocation when no custom `task_id` is passed. 
   
   My point is: this is a function, I expect to be able to call it as many time as I wish. I expect Airflow to treat each call of this function (in proper context) to be as creating a new task.
   
   > You can either do (1) `update_user(i) for i in range(20)` or (2) `update_user >> other_operation`, but not both. I prefer to support 2nd option as it adapts more to what Airflow already does with operators.
   
   Why should I not be able to do this? This is something that I saw many times. 
   ```python
   first_task = BashOperator()
   last_task = BashOperator()
   
   for user_id in users_list:
      first_task >> update_user(user_id) >> last_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.

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



[GitHub] [airflow] aoen commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs

Review comment:
       Should we just drop the kwargs/etc support in the new operator though since it's deprecated?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'

Review comment:
       ```suggestion
       ui_color = PythonOperator. ui_color
   ```
   To keep it consistent. WDYT?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).

Review comment:
       ```suggestion
       # there are some cases we can't deepcopy the objects (e.g protobuf).
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')

Review comment:
       This is tricky one... and I would be in favor of using `inspect.ismethod` instead of argument checking. Otherwise this will rise error:
   ```python
   @task
   def troll_airflow(self):
       return f"Haha it doesn't work"
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)

Review comment:
       Not sure we want to log this. It can be small or big output. At least use debug 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       Is there any particular reason why we don't do simply:
   ```python
   def task(*args, **kwargs):
       """
       Python operator decorator. Wraps a function into an Airflow operator.
       Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
       on function invocation. Use alias to reuse function in the DAG.
       """
       if args:
           raise AirflowException("No args allowed")
       
       def wrapper(f):
           """
           Python wrapper to generate PythonFunctionalOperator out of simple python functions.
           Used for Airflow functional interface
           """
           _PythonFunctionalOperator.validate_python_callable(f)
   
           @functools.wraps(f)
           def factory(*args, **f_kwargs):
               op = _PythonFunctionalOperator(
                   python_callable=f, 
                   task_id=f.__name__,
                   op_args=args, 
                   op_kwargs=f_kwargs, 
                   **kwargs
                )
               return XComArg(op)
           return factory
       return wrapper
   ```
   

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       ```suggestion
           super().__init__(**kwargs)
           self.task_id = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:

Review comment:
       ```suggestion
       def _get_unique_task_id(task_id: str, dag: Optional[DAG] = None) -> str:
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       Would you mind adding a comment about how the auto generated id looks like? It would make it easier to understand what we are doing here. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            self.log.info(f'Returned output was type {type(return_value)} expected dictionary '
+                          'for multiple_outputs')

Review comment:
       As a user I would be confused: does it work or not? We should fail here I think. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.

Review comment:
       ```suggestion
           """
           Validate that python callable can be wrapped by operator.
   ```

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            self.log.info(f'Returned output was type {type(return_value)} expected dictionary '
+                          'for multiple_outputs')
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.

Review comment:
       Is this still valid?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       We can use f-strings 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            self.log.info(f'Returned output was type {type(return_value)} expected dictionary '
+                          'for multiple_outputs')
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool

Review comment:
       This is an unexpected parameter. There's no such param in `task` signature. 




----------------------------------------------------------------
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 pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Great work @casassg πŸŽ‰ 


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Resolving after update passed in mailing 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.

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



[GitHub] [airflow] turbaszek commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Two small comments, otherwise it looks really good! I hope we will be able to merge it soon πŸš€ It would be good to have another look from someone else. @dimberman @feluelle @kaxil @mik-laj 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.

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



[GitHub] [airflow] casassg commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Mostly adding more hours to my day. Sorry, been a bit busy this week. Will try to address most comments


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.

Review comment:
       ```suggestion
   Airflow ``task`` decorator converts any Python function to an Airflow operator.
   The decorated function can be called once to set the arguments and key arguments for operator execution.
   ```

##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Task decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function more than once in a DAG. The decorated function will automatically generate unique
+a ``task_id`` for each generated operator.

Review comment:
       ```suggestion
   You can call a decorated function more than once in a DAG. The decorated function will automatically generate
   a unique ``task_id`` for each generated operator.
   ```

##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')

Review comment:
       ```suggestion
   	with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
   
   		@dag.task
   		def hello_world():
   			print('hello world!')
   
   
   	  # Also...
   
   		from airflow.decorators import task
   
   		@task
   		def hello_name(name: str):
   	    	print(f'hello {name}!')
   
   		hello_name('Airflow users')
   ```

##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Task decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function more than once in a DAG. The decorated function will automatically generate unique
+a ``task_id`` for each generated operator.
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def update_user(user_id: int):
+      ...
+
+    # Avoid generating this list dynamically to keep dag topology stable between DAG runs

Review comment:
       ```suggestion
       # Avoid generating this list dynamically to keep DAG topology stable between DAG runs
   ```




----------------------------------------------------------------
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] boring-cyborg[bot] commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #8962:
URL: https://github.com/apache/airflow/pull/8962#issuecomment-648449772


   Awesome work, congrats on your first merged pull request!
   


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       > We could capture task_id kwarg and generate a new operator, but then what is `update_user` the first operator, the latest one? What does `update_user` represent?
   
   For me `update_user` is a function and as a function it can be called many times with different input thus yielding different results (here creating new task). I have never meet "function as a singleton" pattern. If we don't want to generate `task_id` for users then we may consider raising an exception on second invocation when no custom `task_id` is passed. 
   
   My point is: this is a function, I expect to be able to call it as many time as I wish. I expect Airflow to treat each call of this function (in proper context) as creating a new task.
   
   > You can either do (1) `update_user(i) for i in range(20)` or (2) `update_user >> other_operation`, but not both. I prefer to support 2nd option as it adapts more to what Airflow already does with operators.
   
   Why should I not be able to do this? This is something that I saw many times. 
   ```python
   first_task = BashOperator()
   last_task = BashOperator()
   
   for user_id in users_list:
      first_task >> update_user(user_id) >> last_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.

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



[GitHub] [airflow] casassg commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       With current approach you get both: unrolled and not unrolled. You access each as needed. It's flexible and easier to use when you need several outputs. Also it mimics other frameworks, so it should feel natural for the user.
   
   Resolving comment for now, if someone has anything else to discuss on the topic, please reopen or start new thread.




----------------------------------------------------------------
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] dimberman commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       @casassg I'm a bit confused, can we not give a task_id as an argument and have that become the task_id? Ideally we wouldn't just want "task_1" "task_2" etc. if we can name them (unless I'm missing something which I probably am)




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Can you explain what you mean about "more composable DAGs"? I don't understand what you mean.
   
   The AIP <https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-31%3A+Airflow+functional+DAG+definition> has this:
   
   ```python
     ip_info = get_ip()
     subject = email_subject_generator(ip_info)
     body = email_body_generator(ip_info)
     send_email(subject=subject, html_content=body)
   ```
   
   So this PR doesn't implement what we voted on.  




----------------------------------------------------------------
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] dimberman commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   This is gonna be awesome! Thank you @casassg !


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Does this need any name? Can't we just say that this is a way how users can define DAGs? Personally I would encourage users to use this way instead of using XComs. 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.

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



[GitHub] [airflow] casassg commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),

Review comment:
       Added a more extensive 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.

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



[GitHub] [airflow] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       You are also using F-Strings here, too - not only in examples.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       I see another issue here. Currently, if `task_id` is not provided user will get `KeyError: 'task_id'` instead of `TypeError: __init__() missing 1 required positional argument: 'task_id' `
   




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be

Review comment:
       With current approach you get both: unrolled and not unrolled. You access each as needed. It's flexible and easier to use when you need several outputs. Also it mimics other frameworks, so it should feel natural for the user.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Main worry is that then what is `update_user`. What you are describing here is using `update_user` as an operator factory. It has it's value, but it also feels too magic to me atm. If `update_user` is a factory, then you can't change the operator instance at all or use it to set non-data dependencies. 
   
   We could capture task_id kwarg and generate a new operator, but then what is `update_user` the first operator, the latest one? What does `update_user` represent?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: tests/operators/test_python.py
##########
@@ -312,6 +315,327 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTaskDecorator(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.owner = 'airflow_tes'
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': self.owner,
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_fails_bad_signature(self):
+        """Tests that @task will fail if signature is not binding."""
+        @task_decorator
+        def add_number(num: int) -> int:
+            return num + 2
+        with pytest.raises(TypeError):
+            add_number(2, 3)  # pylint: disable=too-many-function-args
+        with pytest.raises(TypeError):
+            add_number()  # pylint: disable=no-value-for-parameter
+        add_number('test')  # pylint: disable=no-value-for-parameter
+
+    def test_fail_method(self):
+        """Tests that @task will fail if signature is not binding."""
+
+        with pytest.raises(AirflowException):
+            class Test:
+                num = 2
+
+                @task_decorator
+                def add_number(self, num: int) -> int:
+                    return self.num + num
+            Test().add_number(2)
+
+    def test_fail_multiple_outputs_key_type(self):
+        @task_decorator(multiple_outputs=True)
+        def add_number(num: int):
+            return {2: num}
+        with self.dag:
+            ret = add_number(2)
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        with pytest.raises(AirflowException):
+            # pylint: disable=maybe-no-member
+            ret.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+    def test_fail_multiple_outputs_no_dict(self):
+        @task_decorator(multiple_outputs=True)
+        def add_number(num: int):
+            return num
+
+        with self.dag:
+            ret = add_number(2)
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+
+        with pytest.raises(AirflowException):
+            # pylint: disable=maybe-no-member
+            ret.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        ret = task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        ret.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)  # pylint: disable=maybe-no-member
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        ret = task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        ret.operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)  # pylint: disable=maybe-no-member
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_manual_task_id(self):
+        """Test manually seting task_id"""
+
+        @task_decorator(task_id='some_name')
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['some_name'] == self.dag.task_ids
+
+    def test_multiple_calls(self):
+        """Test calling task multiple times in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run()
+            do_run_2 = do_run()
+            assert ['do_run', 'do_run__1', 'do_run__2'] == self.dag.task_ids
+
+        assert do_run_1.operator.task_id == 'do_run__1'  # pylint: disable=maybe-no-member
+        assert do_run_2.operator.task_id == 'do_run__2'  # pylint: disable=maybe-no-member
+
+    def test_call_20(self):
+        """Test calling decorated function 10 times in a DAG"""

Review comment:
       ```suggestion
           """Test calling decorated function 21 times in a DAG"""
   ```

##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)

Review comment:
       We've already got `TestPythonBase` as a base class -- inherit from that instead of duplicating these functions again please.

##########
File path: docs/concepts.rst
##########
@@ -173,6 +214,60 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+*Added in Airflow 1.10.11*
+
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+If you call a decorated function twice in a DAG, it will error as it doesn't know what args to use.
+If you want to reuse decorated functions, use the copy method as follows:

Review comment:
       I don't think this is the case anymore, right?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.

Review comment:
       that `XCom values`, then SGTM.




----------------------------------------------------------------
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] evgenyshulman commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,144 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        _old_kwargs = self._kwargs
+        if task_id:
+            _old_kwargs['task_id'] = task_id

Review comment:
       [minor] there is a side affect on self._kwargs, it doesn't break anything now but can lead to problems in the future.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)

Review comment:
       Sure. reverted to use `TestPythonBase`




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,144 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        _old_kwargs = self._kwargs
+        if task_id:
+            _old_kwargs['task_id'] = task_id

Review comment:
       See also https://github.com/apache/airflow/pull/8962#discussion_r429168181




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       Ahhh gotcha. I hadn't noticed/appreciated the difference between `ip_info = get_ip()` and `get_ip.output` and was merging them together in my head.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,145 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)

Review comment:
       ```suggestion
       :param op_kwargs: a dictionary of keyword arguments that will get unpacked
           in your function (templated)
       :type op_kwargs: dict
       :param op_args: a list of positional arguments that will get unpacked when
           calling your callable (templated)
       :type op_args: 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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       The question is do we want to backport 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.

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



[GitHub] [airflow] feluelle commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       Or we could use [future-fstrings](https://github.com/asottile/future-fstrings). We already have it as dep of a [dep](https://github.com/apache/airflow/search?q=future-fstrings&unscoped_q=future-fstrings) and add `# -*- coding: future_fstrings -*-`.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,145 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :type op_kwargs: dict
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :type op_args: list
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = PythonOperator.ui_color
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        task_id: str,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(task_id, kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG] = None) -> str:
+        """
+        Generate unique task id given a DAG (or if run in a DAG context)
+        Ids are generated by appending a unique number to the end of
+        the original task id.
+
+        Example:
+          task_id
+          task_id__1
+          task_id__2
+          ...
+          task_id__20
+        """
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """
+        Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.debug("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            raise AirflowException(f'Returned output was type {type(return_value)} expected dictionary '
+                                   'for multiple_outputs')
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, multiple_outputs: bool = False, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Can be reused in a single DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as XCom keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """
+        Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        _PythonFunctionalOperator.validate_python_callable(f)
+        kwargs['task_id'] = kwargs.get('task_id', None) or f.__name__

Review comment:
       Unimportant change but:
   
   ```suggestion
           kwargs.setdefault('task_id', f.__name__)
   ```




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,61 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned

Review comment:
       Did you mean task, not operator?
   ```suggestion
   Task decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
   ```




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: tests/operators/test_python.py
##########
@@ -311,6 +315,350 @@ def func(**context):
         python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
 
 
+class TestAirflowTask(unittest.TestCase):
+
+    @classmethod
+    def setUpClass(cls):
+        super().setUpClass()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def setUp(self):
+        super().setUp()
+        self.dag = DAG(
+            'test_dag',
+            default_args={
+                'owner': 'airflow',
+                'start_date': DEFAULT_DATE})
+        self.addCleanup(self.dag.clear)
+
+    def tearDown(self):
+        super().tearDown()
+
+        with create_session() as session:
+            session.query(DagRun).delete()
+            session.query(TI).delete()
+
+    def _assert_calls_equal(self, first, second):
+        assert isinstance(first, Call)
+        assert isinstance(second, Call)
+        assert first.args == second.args
+        # eliminate context (conf, dag_run, task_instance, etc.)
+        test_args = ["an_int", "a_date", "a_templated_string"]
+        first.kwargs = {
+            key: value
+            for (key, value) in first.kwargs.items()
+            if key in test_args
+        }
+        second.kwargs = {
+            key: value
+            for (key, value) in second.kwargs.items()
+            if key in test_args
+        }
+        assert first.kwargs == second.kwargs
+
+    def test_python_operator_python_callable_is_callable(self):
+        """Tests that @task will only instantiate if
+        the python_callable argument is callable."""
+        not_callable = {}
+        with pytest.raises(AirflowException):
+            task_decorator(not_callable, dag=self.dag)
+
+    def test_python_callable_arguments_are_templatized(self):
+        """Test @task op_args are templatized"""
+        recorded_calls = []
+
+        # Create a named tuple and ensure it is still preserved
+        # after the rendering is done
+        Named = namedtuple('Named', ['var1', 'var2'])
+        named_tuple = Named('{{ ds }}', 'unchanged')
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag)
+        task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
+
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        ds_templated = DEFAULT_DATE.date().isoformat()
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(4,
+                 date(2019, 1, 1),
+                 "dag {} ran on {}.".format(self.dag.dag_id, ds_templated),
+                 Named(ds_templated, 'unchanged'))
+        )
+
+    def test_python_callable_keyword_arguments_are_templatized(self):
+        """Test PythonOperator op_kwargs are templatized"""
+        recorded_calls = []
+
+        task = task_decorator(
+            # a Mock instance cannot be used as a callable function or test fails with a
+            # TypeError: Object of type Mock is not JSON serializable
+            build_recording_function(recorded_calls),
+            dag=self.dag
+        )
+        task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
+        self.dag.create_dagrun(
+            run_id=DagRunType.MANUAL.value,
+            execution_date=DEFAULT_DATE,
+            start_date=DEFAULT_DATE,
+            state=State.RUNNING
+        )
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
+        assert len(recorded_calls) == 1
+        self._assert_calls_equal(
+            recorded_calls[0],
+            Call(an_int=4,
+                 a_date=date(2019, 1, 1),
+                 a_templated_string="dag {} ran on {}.".format(
+                     self.dag.dag_id, DEFAULT_DATE.date().isoformat()))
+        )
+
+    def test_copy_in_dag(self):
+        """Test copy method to reuse tasks in a DAG"""
+
+        @task_decorator
+        def do_run():
+            return 4
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1 = do_run.copy()
+            do_run_2 = do_run.copy()
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+
+    def test_copy(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def do_run():
+            return 4
+
+        @task_decorator
+        def do__run():
+            return 4
+        do_run_1 = do_run.copy()
+        do_run_2 = do_run.copy()
+        do__run_1 = do__run.copy()
+        do__run_2 = do__run.copy()
+        with self.dag:
+            do_run()
+            assert ['do_run'] == self.dag.task_ids
+            do_run_1()
+            do_run_2()
+            do__run()
+            do__run_1()
+            do__run_2()
+
+        assert do_run_1.task_id == 'do_run__1'
+        assert do_run_2.task_id == 'do_run__2'
+        assert do__run_1.task_id == 'do__run__1'
+        assert do__run_2.task_id == 'do__run__2'
+
+    def test_copy_10(self):
+        """Test copy method outside of a DAG"""
+        @task_decorator
+        def __do_run():
+            return 4
+
+        with self.dag:
+            __do_run()
+            do_runs = [__do_run.copy() for _ in range(20)]
+
+        assert do_runs[-1].task_id == '__do_run__20'
+
+    def test_dict_outputs(self):
+        """Tests pushing multiple outputs as a dictionary"""
+
+        @task_decorator(multiple_outputs=True)
+        def return_dict(number: int):
+            return {
+                'number': number + 1,
+                43: 43

Review comment:
       Yes, I mean we should define, and test, what we want the behaviour to be in this case.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       > We could capture task_id kwarg and generate a new operator, but then what is `update_user` the first operator, the latest one? What does `update_user` represent?
   
   For me `update_user` is a function and as a function it can be called many times with different input thus yielding different results. I have never meet "function as a singleton" pattern. If we don't want to generate `task_id` for users then we may consider raising an exception on second invocation when no custom `task_id` is passed. 
   
   My point is: this is a function, I expect to be able to call it as many time as I wish. I expect Airflow to treat each call of this function (in proper context) as creating a new task.
   
   > You can either do (1) `update_user(i) for i in range(20)` or (2) `update_user >> other_operation`, but not both. I prefer to support 2nd option as it adapts more to what Airflow already does with operators.
   
   Why should I not be able to do this? This is something that I saw many times. 
   ```python
   first_task = BashOperator()
   last_task = BashOperator()
   
   for user_id in users_list:
      first_task >> update_user(user_id) >> last_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.

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



[GitHub] [airflow] ashb commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,61 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function twice in a DAG. The decorated function will automatically generate unique a ``task_id``
+for each generated operator.
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def update_user(user_id: int):
+      ...
+
+    for user_id in user_ids:

Review comment:
       Do we need to add a note here with caveats about "don't make the list dynamic/quickly changing"?




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       This will not work. If I pass arguments to `@task(multiple_outputs=True)` this will break.




----------------------------------------------------------------
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 pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Do copy what I put in slack, Thanks to Stackoverflow I've got a way of having `from airflow import task` be callable, and still work as a submodule:
   
   ```diff
   diff --git airflow/task/__init__.py airflow/task/__init__.py
   index 114d189da..0b15c9a8c 100644
   --- airflow/task/__init__.py
   +++ airflow/task/__init__.py
   @@ -16,3 +16,20 @@
    # KIND, either express or implied.  See the License for the
    # specific language governing permissions and limitations
    # under the License.
   +
   +import sys
   +import types
   +
   +
   +class CallableModule(types.ModuleType):
   +    def __init__(self):
   +        types.ModuleType.__init__(self, __name__)
   +        self.__dict__.update(sys.modules[__name__].__dict__)
   +
   +    __all__ = list(set(vars().keys()) - {'__qualname__'})   # for python 2 and 3
   +
   +    def __call__(self, *args, **kwargs):
   +        from airlfow.decorators import task
   +        return task(*args, **kwargs)
   +
   +sys.modules[__name__] = CallableModule()
   ```
   
   This works in Py 2.7, 3.7 and 3.8 (versions I have easy access to).
   
   The main question is: do we want to support that? Is this "hack" worth it, and do IDEs get massively confused by 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.

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



[GitHub] [airflow] casassg commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')

Review comment:
       Seems `ismethod` does not work. https://stackoverflow.com/questions/47599749/check-if-function-belongs-to-a-class, keeping it as is as the __qualname__ may involve a bit more difficult setup and does not accept functions defined in classes that are not methods.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            self.log.info(f'Returned output was type {type(return_value)} expected dictionary '
+                          'for multiple_outputs')

Review comment:
       Sure, repaced and added test.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)

Review comment:
       Sg!

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       Added!

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       Yes, this code will fail when you do:
   ```
   @task 
   def example(...)
     ...
   ``` 
   decorators get function as the first arg. If we want to allow both kwargs and no kwargs we need to do this small and more verbose approach.
   
   

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       Why is this preferred?

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key in return_value.keys():
+                if not isinstance(key, str):
+                    raise AirflowException('Returned dictionary keys must be strings when using '
+                                           f'multiple_outputs, found {key} ({type(key)}) instead')
+            for key, value in return_value.items():
+                self.xcom_push(context, key, value)
+        else:
+            self.log.info(f'Returned output was type {type(return_value)} expected dictionary '
+                          'for multiple_outputs')
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool

Review comment:
       Added it here for visibility.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       Yeah, we can use f-strings here, and manually fix it up when backporting -- it's not too much work on the release-manager/who ever does the backport.
   
   When it comes time to backport this I can run someone through the process (or better yet, I should document this process)




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')

Review comment:
       ```suggestion
     with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
         @dag.task
         def hello_world():
             print('hello world!')
     
     
         # Also...
         from airflow.decorators import task
     
     
         @task
         def hello_name(name: str):
             print(f'hello {name}!')
     
     
         hello_name('Airflow users')
   ```




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.

Review comment:
       I'm open to other options. This seemed the cleanest one. We are basically calling __init__ again so that we capture default_args from the dag. The other option is to manually implement default_args here. 
   
   Also note that we will get parse errors in the declaration if there's any. So this will be for default_args itself. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()

Review comment:
       `dag` -> because we may declare the task outside without an explicit dag (and we want to fallback to current_dag then. 
   `task_id` ->  not really any reason. can switch it. 

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _fail_if_method(python_callable):
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:

Review comment:
       Mainly is bc of 1 function == 1 operator. The alternative was the idea that @evgenyshulman proposed which is to use functions as operators generators. This simplifies a bit, but also then you can't use the change the operator later on (it's never accessible in the DAG file itself). I've seen both approaches take here (either 1to1 or 1tomany). For me I like better the 1to1, but mainly bc it allows you to use it as you would use any operator later and feels a bit more intuitive.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()

Review comment:
       DAG can be not specified when initialized outside of DAG context. And you want to make it to bind when you call it. If you manually assign it the DAG, then it wont work thats true. Not sure if I should check this here.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '
+                                   'multiple_outputs=True.')
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        self._fail_if_method(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs

Review comment:
       This already calls `BaseOperator.__init__` so guessing it will already give you a warning.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:
+            raise AirflowException('@task needs to have either do_xcom_push=True or '

Review comment:
       Actually, I may remove this code. I can think of tasks that do not need to push XCom and we should still allow that.

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +147,141 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        # Check if we need to generate a new task_id
+        task_id = kwargs.get('task_id', None)
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        if task_id and dag and task_id in dag.task_ids:
+            prefix = task_id.rsplit("__", 1)[0]
+            task_id = sorted(
+                filter(lambda x: x.startswith(prefix), dag.task_ids),
+                reverse=True
+            )[0]
+            num = int(task_id[-1] if '__' in task_id else '0') + 1
+            kwargs['task_id'] = f'{prefix}__{num}'
+
+        if not kwargs.get('do_xcom_push', True) and not multiple_outputs:

Review comment:
       Default is already set. removing code.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+
+        # Check that arguments can be binded
+        signature(python_callable).bind(*op_args, **op_kwargs)
+        self.multiple_outputs = multiple_outputs
+        self.op_args = op_args
+        self.op_kwargs = op_kwargs
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def validate_python_callable(python_callable):
+        """Validate that python callable can be wrapped by operator.
+        Raises exception if invalid.
+
+        :param python_callable: Python object to be validated
+        :raises: TypeError, AirflowException
+        """
+        if not callable(python_callable):
+            raise TypeError('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')

Review comment:
       It will fail as `@classmethod` does not return a callable it seems (surprising to me as well, but it fails on checking if python_callable is a callable).

##########
File path: airflow/operators/python.py
##########
@@ -145,6 +149,131 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function
+    :type op_kwargs: dict (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable
+    :type op_args: list (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('op_args', 'op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        op_args: Tuple[Any],
+        op_kwargs: Dict[str, Any],
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], kwargs.get('dag', None))
+        super().__init__(**kwargs)

Review comment:
       Nvm, seems this proposed change fails due to task_id being checked if repeated on ` super().__init__(**kwargs)`
   
   Resolving.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Passed +5 (bindings) +2 (non-binding). No opposition to the update in the AIP.
   
   


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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>'

Review comment:
       f-strings are not a blocker and we are using them in many places 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   1. Switched to use factory method (aka reuse function several times in a DAG). Cleaned a lot of code that had to do with this decision.
   2. Added check for multiple outputs keys. Now if any key is not a string, it will fail. This reduces weird cases where users may accidentally duplicate keys when casted to strings. Less flexible but also easier to see what's happenning below on the XCom table. 
   3. Reverted string casting in XComArg as we no longer need as we are enforcing keys to always be strings


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Definitely! We should make an issue for that, discuss and then adjust the docs in a follow-up PR




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/models/xcom_arg.py
##########
@@ -83,7 +83,7 @@ def __getitem__(self, item):
         """
         Implements xcomresult['some_result_key']
         """
-        return XComArg(operator=self.operator, key=item)
+        return XComArg(operator=self.operator, key=str(item))

Review comment:
       Key will always be a string as it's casted when pushing the field to the XCom DB. This is just to make things easier to use when doing multiple_outputs also to ensure consistency. This allows you to do:
   ```
   res[1]
   ```
   Which in reality is more transparent.
   
   It was a change we missed in #8652




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       Fair. Main issue is that it does introduce a new syntax so for backwards compatible issues I would add a name to this.
   
   We can switch all documentation to use this style on 2.0 and therefore remove this specific section (or collapse 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.

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



[GitHub] [airflow] ashb commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -173,6 +213,61 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
 We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
 When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.
 
+.. _concepts:task_decorator:
+
+Python task decorator
+---------------------
+
+Airflow ``task`` decorator converts any Python decorated function to a Python Airflow operator.
+The decorated function can be called once to set the arguments and key arguments for operator execution.
+
+
+.. code:: python
+
+  with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
+
+    @dag.task
+    def hello_world():
+      print('hello world!')
+
+
+    # Also...
+
+    from airflow.decorators import task
+
+    @task
+    def hello_name(name: str):
+      print(f'hello {name}!')
+
+    hello_name('Airflow users')
+
+Operator decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
+value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
+lists or tuples into seprate XCom values. This can be used with regular operators to create
+:ref:`functional DAGs <concepts:functional_dags>`.
+
+Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
+operators.
+
+You can call a decorated function twice in a DAG. The decorated function will automatically generate unique a ``task_id``
+for each generated operator.

Review comment:
       ```suggestion
   You can call a decorated function more than once in a DAG. The decorated function will automatically generate unique 
   a ``task_id`` for each generated operator.
   ```




----------------------------------------------------------------
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 edited a comment on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on pull request #8962:
URL: https://github.com/apache/airflow/pull/8962#issuecomment-643526181


   Can you please rebase the PR on the master, there were some CI issues we fixed


----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,47 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs
+---------------
+*Added in Airflow 1.10.11*
+
+DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
+:ref:`XComs <concepts:xcom>` values. In addition, you can wrap functions as tasks using the
+:ref:`task decorator <concepts:task_decorator>`. Dependencies are automatically inferred from
+the message dependencies.
+
+Example DAG with functional abstraction
+
+.. code:: python
+
+  with DAG(
+      'send_server_ip', default_args=default_args, schedule_interval=None
+  ) as dag:
+
+    # 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
+    )
+
+    @dag.task(multiple_outputs=True)
+    def prepare_email(raw_json: 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)

Review comment:
       The `output` part was implemented already as part of a previous PR. This PR focuses on `@task` only. 
   
   The main difference between the proposed approach vs current approach is that in the current approach  we do not need to `call` normal operators. This should reduce complexity on creating functional DAGs. 
   
   In addition, existing operators may have templated fields that are mandatory (example `subject` is required in `EmailOperator` initialization). This basically means that we need to overwrite them on `call` which is a bit confusing (see example below or example on AIP)
   
   The `output` is basically a way to access XComArg for non callable operators. This allows a more smooth definition of functional DAGs between normal operators and `@task` operators.
   
   Old example:
   ```python
       get_ip = SimpleHttpOperator(
           task_id='get_ip', endpoint='get', method='GET', xcom_push=True
       )
   
       @dag.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>'
         }
       server_info = get_ip()
       email_info = prepare_email(server_info)
         
       send_email = EmailOperator(
           task_id='send_email',
           to='example@example.com',
           subject='',
           html_content=''
       )
       send_email(subject=email_info['subject'], html_content=email_info['body'])
   ```
   
   New approach:
   ```python
   
       get_ip = SimpleHttpOperator(
           task_id='get_ip', endpoint='get', method='GET', xcom_push=True
       )
   
       @dag.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='example@example.com',
           subject=email_info['subject'],
           html_content=email_info['body']
       )
   ```
   
   I do believe the new approach is better. I'm happy to change the AIP and submit it for vote if that seems something that may be required.




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,142 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        *args,
+        **kwargs
+    ) -> None:
+        dag = kwargs.get('dag', None) or DagContext.get_current_dag()
+        kwargs['task_id'] = self._get_unique_task_id(kwargs['task_id'], dag)
+        self._validate_python_callable(python_callable)
+        super().__init__(*args, **kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id, dag):
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'
+
+    @staticmethod
+    def _validate_python_callable(python_callable):
+        if not callable(python_callable):
+            raise AirflowException('`python_callable` param must be callable')
+        if 'self' in signature(python_callable).parameters.keys():
+            raise AirflowException('@task does not support methods')
+
+    def __call__(self, *args, **kwargs):
+        # If args/kwargs are set, then operator has been called. Raise exception
+        if self._called:
+            raise AirflowException('@task decorated functions can only be called once. If you need to reuse '
+                                   'it several times in a DAG, use the `copy` method.')
+
+        # If we have no DAG, reinitialize class to capture DAGContext and DAG default args.
+        if not self.has_dag():
+            self.__init__(python_callable=self.python_callable,
+                          multiple_outputs=self.multiple_outputs,
+                          **self._kwargs)
+
+        # Capture args/kwargs
+        self._op_args = args
+        self._op_kwargs = kwargs
+        self._called = True
+        return XComArg(self)
+
+    def copy(self, task_id: Optional[str] = None, **kwargs):
+        """
+        Create a copy of the task, allow to overwrite ctor kwargs if needed.
+
+        If alias is created a new DAGContext, apply defaults and set new DAG as the operator DAG.
+
+        :param task_id: Task id for the new operator
+        :type task_id: Optional[str]
+        """
+        if task_id:
+            self._kwargs['task_id'] = task_id
+        return _PythonFunctionalOperator(
+            python_callable=self.python_callable,
+            multiple_outputs=self.multiple_outputs,
+            **{**kwargs, **self._kwargs}
+        )
+
+    def execute(self, context: Dict):
+        return_value = self.python_callable(*self._op_args, **self._op_kwargs)
+        self.log.info("Done. Returned value was: %s", return_value)
+        if not self.multiple_outputs:
+            return return_value
+        if isinstance(return_value, dict):
+            for key, value in return_value.items():
+                self.xcom_push(context, str(key), value)
+        elif isinstance(return_value, (list, tuple)):
+            for key, value in enumerate(return_value):
+                self.xcom_push(context, str(key), value)
+        return return_value
+
+
+def task(python_callable: Optional[Callable] = None, **kwargs):
+    """
+    Python operator decorator. Wraps a function into an Airflow operator.
+    Accepts kwargs for operator kwarg. Will try to wrap operator into DAG at declaration or
+    on function invocation. Use alias to reuse function in the DAG.
+
+    :param python_callable: Function to decorate
+    :type python_callable: Optional[Callable]
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. List/Tuples will unroll to xcom values
+        with index as key. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+
+    """
+    def wrapper(f):
+        """Python wrapper to generate PythonFunctionalOperator out of simple python functions.
+        Used for Airflow functional interface
+        """
+        return _PythonFunctionalOperator(python_callable=f, task_id=f.__name__, **kwargs)
+    if callable(python_callable):
+        return wrapper(python_callable)
+    elif python_callable is not None:
+        raise AirflowException('No args allowed while using @task, use kwargs instead')
+    return wrapper

Review comment:
       What about?
   ```python
   if len(args) > 1:
           raise AirflowException("No args allowed")
   ```
   We do this in `apply_defaults`. 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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



##########
File path: docs/concepts.rst
##########
@@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
 
     op.dag is dag # True
 
+.. _concepts:functional_dags:
+
+Functional DAGs

Review comment:
       @casassg do we have any backwards issues other than metaclass? 




----------------------------------------------------------------
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 #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   @casassg flake8 is sad, if you wish I can 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.

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



[GitHub] [airflow] casassg commented on pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

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


   Yay! Thanks everyone for the patience and through review πŸŽ‰ 


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