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 23:30:13 UTC

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

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