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:
us...@infra.apache.org


Reply via email to