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


Reply via email to