From bbc725f8dcd981b13240b9efaf8a7ed13d4d38b2 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Fri, 26 Jan 2024 11:58:15 +0100 Subject: [PATCH 01/11] feature: callable for template_fields --- airflow/models/baseoperator.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 98532d90b0256..3d02a838872b0 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1361,6 +1361,11 @@ def render_template_fields( :param context: Context dict with values to apply on content. :param jinja_env: Jinja's environment to use for rendering. """ + for name in self.template_fields: + attribute = getattr(self, name) + if callable(attribute): + setattr(self, name, attribute()) + if not jinja_env: jinja_env = self.get_template_env() self._do_render_template_fields(self, self.template_fields, context, jinja_env, set()) From e4af4f43fe86c4f3aba3f31510a0c5e3128d5879 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Thu, 21 Mar 2024 00:05:37 +0100 Subject: [PATCH 02/11] review_1: add test --- tests/models/test_baseoperator.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index 8513d2ebb05be..cf1a6e2c46861 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -358,6 +358,19 @@ def test_render_template_fields(self): assert task.arg1 == "footemplated" assert task.arg2 == "bartemplated" + @pytest.mark.db_test + def test_render_template_fields_func(self): + """Verify if operator attributes are correctly templated.""" + + def fn_to_template(): + return "{{ bar }}" + + task = MockOperator(task_id="op1", arg2=fn_to_template) + + # Trigger templating and verify if attributes are templated correctly + task.render_template_fields(context={"bar": "bartemplated"}) + assert task.arg2 == "bartemplated" + @pytest.mark.parametrize(("content",), [(object(),), (uuid.uuid4(),)]) def test_render_template_fields_no_change(self, content): """Tests if non-templatable types remain unchanged.""" From d44830b24d0463753e348dfd3cdcd697014129f2 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Thu, 30 May 2024 16:09:45 +0200 Subject: [PATCH 03/11] review: 2 --- airflow/models/baseoperator.py | 19 ++++++++++++++++--- tests/models/test_baseoperator.py | 7 ++++--- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 3d02a838872b0..9d9a6f35121c2 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1349,26 +1349,39 @@ def __getstate__(self): def __setstate__(self, state): self.__dict__ = state + def render_value(self, context, jinja_env, value): + return self.render_template( + value, + context, + jinja_env, + set(), + ) + def render_template_fields( self, context: Context, jinja_env: jinja2.Environment | None = None, ) -> None: - """Template all attributes listed in *self.template_fields*. + """Template all attributes listed in *self.template_fields* except for callable. This mutates the attributes in-place and is irreversible. :param context: Context dict with values to apply on content. :param jinja_env: Jinja's environment to use for rendering. """ + template_fields_to_render = [] + + render_fn = functools.partial(self.render_value, context, jinja_env) for name in self.template_fields: attribute = getattr(self, name) if callable(attribute): - setattr(self, name, attribute()) + setattr(self, name, attribute(render_fn)) + else: + template_fields_to_render.append(name) if not jinja_env: jinja_env = self.get_template_env() - self._do_render_template_fields(self, self.template_fields, context, jinja_env, set()) + self._do_render_template_fields(self, template_fields_to_render, context, jinja_env, set()) @provide_session def clear( diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index cf1a6e2c46861..4e2c334ce8967 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -362,14 +362,15 @@ def test_render_template_fields(self): def test_render_template_fields_func(self): """Verify if operator attributes are correctly templated.""" - def fn_to_template(): - return "{{ bar }}" + def fn_to_template(render_fn): + tmp = render_fn("{{ bar }}") + return "foo_" + tmp task = MockOperator(task_id="op1", arg2=fn_to_template) # Trigger templating and verify if attributes are templated correctly task.render_template_fields(context={"bar": "bartemplated"}) - assert task.arg2 == "bartemplated" + assert task.arg2 == "foo_bartemplated" @pytest.mark.parametrize(("content",), [(object(),), (uuid.uuid4(),)]) def test_render_template_fields_no_change(self, content): From fa07e94366a16d86c7073314637bbd423bfd6d9b Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Mon, 3 Jun 2024 15:24:07 +0200 Subject: [PATCH 04/11] review 3 --- airflow/models/abstractoperator.py | 25 +++++++++++++++++++------ airflow/models/baseoperator.py | 20 +------------------- 2 files changed, 20 insertions(+), 25 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index b7160430e066a..bd15b29bb831b 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -18,6 +18,7 @@ from __future__ import annotations import datetime +import functools import inspect from abc import abstractproperty from functools import cached_property @@ -696,6 +697,14 @@ def get_template_env(self, dag: DAG | None = None) -> jinja2.Environment: dag = self.get_dag() return super().get_template_env(dag=dag) + def render_value(self, context: Context, jinja_env: jinja2.Environment, value): + return self.render_template( + value, + context, + jinja_env, + set(), + ) + def _do_render_template_fields( self, parent: Any, @@ -705,6 +714,7 @@ def _do_render_template_fields( seen_oids: set[int], ) -> None: """Override the base to use custom error logging.""" + render_fn = functools.partial(self.render_value, context, jinja_env) for attr_name in template_fields: try: value = getattr(parent, attr_name) @@ -730,12 +740,15 @@ def _do_render_template_fields( pass try: - rendered_content = self.render_template( - value, - context, - jinja_env, - seen_oids, - ) + if callable(value): + rendered_content = value(render_fn) + else: + rendered_content = self.render_template( + value, + context, + jinja_env, + seen_oids, + ) except Exception: value_masked = redact(name=attr_name, value=value) self.log.exception( diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 9d9a6f35121c2..814fa52ac4a03 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1349,14 +1349,6 @@ def __getstate__(self): def __setstate__(self, state): self.__dict__ = state - def render_value(self, context, jinja_env, value): - return self.render_template( - value, - context, - jinja_env, - set(), - ) - def render_template_fields( self, context: Context, @@ -1369,19 +1361,9 @@ def render_template_fields( :param context: Context dict with values to apply on content. :param jinja_env: Jinja's environment to use for rendering. """ - template_fields_to_render = [] - - render_fn = functools.partial(self.render_value, context, jinja_env) - for name in self.template_fields: - attribute = getattr(self, name) - if callable(attribute): - setattr(self, name, attribute(render_fn)) - else: - template_fields_to_render.append(name) - if not jinja_env: jinja_env = self.get_template_env() - self._do_render_template_fields(self, template_fields_to_render, context, jinja_env, set()) + self._do_render_template_fields(self, self.template_fields, context, jinja_env, set()) @provide_session def clear( From b3a83713b7cd423f62a8e34867b86f15c10b58a5 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Tue, 4 Jun 2024 10:11:11 +0200 Subject: [PATCH 05/11] review 4 --- airflow/models/abstractoperator.py | 11 ++--------- tests/models/test_baseoperator.py | 18 +++++++++++++++--- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index bd15b29bb831b..63282329d91c5 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import functools import inspect from abc import abstractproperty from functools import cached_property @@ -698,12 +697,7 @@ def get_template_env(self, dag: DAG | None = None) -> jinja2.Environment: return super().get_template_env(dag=dag) def render_value(self, context: Context, jinja_env: jinja2.Environment, value): - return self.render_template( - value, - context, - jinja_env, - set(), - ) + return self.render_template(value, context, jinja_env) def _do_render_template_fields( self, @@ -714,7 +708,6 @@ def _do_render_template_fields( seen_oids: set[int], ) -> None: """Override the base to use custom error logging.""" - render_fn = functools.partial(self.render_value, context, jinja_env) for attr_name in template_fields: try: value = getattr(parent, attr_name) @@ -741,7 +734,7 @@ def _do_render_template_fields( try: if callable(value): - rendered_content = value(render_fn) + rendered_content = value(self.render_value, context, jinja_env) else: rendered_content = self.render_template( value, diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index 4e2c334ce8967..0330389774ab3 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -359,11 +359,11 @@ def test_render_template_fields(self): assert task.arg2 == "bartemplated" @pytest.mark.db_test - def test_render_template_fields_func(self): + def test_render_template_fields_func_using_context(self): """Verify if operator attributes are correctly templated.""" - def fn_to_template(render_fn): - tmp = render_fn("{{ bar }}") + def fn_to_template(render_value_fn, context: Context, jinja_env): + tmp = render_value_fn(context, jinja_env, "{{ bar }}") return "foo_" + tmp task = MockOperator(task_id="op1", arg2=fn_to_template) @@ -372,6 +372,18 @@ def fn_to_template(render_fn): task.render_template_fields(context={"bar": "bartemplated"}) assert task.arg2 == "foo_bartemplated" + @pytest.mark.db_test + def test_render_template_fields_simple_func(self): + """Verify if operator attributes are correctly templated.""" + + def fn_to_template(*args): + a = "foo_" + ("bar" * 3) + return a + + task = MockOperator(task_id="op1", arg2=fn_to_template) + task.render_template_fields({}) + assert task.arg2 == "foo_barbarbar" + @pytest.mark.parametrize(("content",), [(object(),), (uuid.uuid4(),)]) def test_render_template_fields_no_change(self, content): """Tests if non-templatable types remain unchanged.""" From 1fb9a07c52ff2d0dbc8bbef4d6fc88edba7cb549 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Tue, 4 Jun 2024 11:23:46 +0200 Subject: [PATCH 06/11] review 5 --- airflow/models/abstractoperator.py | 2 +- tests/models/test_baseoperator.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index 63282329d91c5..a04ada2071fd7 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -734,7 +734,7 @@ def _do_render_template_fields( try: if callable(value): - rendered_content = value(self.render_value, context, jinja_env) + rendered_content = value(context=context, jinja_env=jinja_env) else: rendered_content = self.render_template( value, diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index 0330389774ab3..748b9076a4d21 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -362,21 +362,21 @@ def test_render_template_fields(self): def test_render_template_fields_func_using_context(self): """Verify if operator attributes are correctly templated.""" - def fn_to_template(render_value_fn, context: Context, jinja_env): - tmp = render_value_fn(context, jinja_env, "{{ bar }}") + def fn_to_template(context, jinja_env): + tmp = context["task"].render_value(context, jinja_env, "{{ bar }}") return "foo_" + tmp task = MockOperator(task_id="op1", arg2=fn_to_template) # Trigger templating and verify if attributes are templated correctly - task.render_template_fields(context={"bar": "bartemplated"}) + task.render_template_fields(context={"bar": "bartemplated", "task": task}) assert task.arg2 == "foo_bartemplated" @pytest.mark.db_test def test_render_template_fields_simple_func(self): """Verify if operator attributes are correctly templated.""" - def fn_to_template(*args): + def fn_to_template(**kwargs): a = "foo_" + ("bar" * 3) return a From 1b83c3503bf374e6b9759f11ba697ee037c917b5 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Tue, 4 Jun 2024 13:29:33 +0200 Subject: [PATCH 07/11] review 6 --- airflow/models/abstractoperator.py | 3 --- airflow/models/baseoperator.py | 2 +- tests/models/test_baseoperator.py | 2 +- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index a04ada2071fd7..7f713d6d3c58f 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -696,9 +696,6 @@ def get_template_env(self, dag: DAG | None = None) -> jinja2.Environment: dag = self.get_dag() return super().get_template_env(dag=dag) - def render_value(self, context: Context, jinja_env: jinja2.Environment, value): - return self.render_template(value, context, jinja_env) - def _do_render_template_fields( self, parent: Any, diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 814fa52ac4a03..98532d90b0256 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1354,7 +1354,7 @@ def render_template_fields( context: Context, jinja_env: jinja2.Environment | None = None, ) -> None: - """Template all attributes listed in *self.template_fields* except for callable. + """Template all attributes listed in *self.template_fields*. This mutates the attributes in-place and is irreversible. diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index 748b9076a4d21..06310cd395ca0 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -363,7 +363,7 @@ def test_render_template_fields_func_using_context(self): """Verify if operator attributes are correctly templated.""" def fn_to_template(context, jinja_env): - tmp = context["task"].render_value(context, jinja_env, "{{ bar }}") + tmp = context["task"].render_template("{{ bar }}", context, jinja_env) return "foo_" + tmp task = MockOperator(task_id="op1", arg2=fn_to_template) From f6c93b54ff6ac33fa02bdac90296354c87888c26 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Tue, 4 Jun 2024 14:02:17 +0200 Subject: [PATCH 08/11] doc --- .../apache-airflow/core-concepts/operators.rst | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/apache-airflow/core-concepts/operators.rst b/docs/apache-airflow/core-concepts/operators.rst index 0330ef43fbee9..2e8db0099092c 100644 --- a/docs/apache-airflow/core-concepts/operators.rst +++ b/docs/apache-airflow/core-concepts/operators.rst @@ -86,6 +86,24 @@ For example, say you want to pass the start of the data interval as an environme Here, ``{{ ds }}`` is a templated variable, and because the ``env`` parameter of the ``BashOperator`` is templated with Jinja, the data interval's start date will be available as an environment variable named ``DATA_INTERVAL_START`` in your Bash script. +Same for the parameter ``bash_command`` that is also templated with Jinja. +.. code-block:: python + + t = BashOperator(task_id="simple_templated_echo", bash_command="echo {{ ds }}", dag=dag) + +You can also perform templating manually by giving a callable when the value is more complex and/or dynamic + +.. code-block:: python + + def build_complex_command(context, jinja_env): + with open("file.csv") as f: + data = do_complex_things(f) + data.replace("THE_DATE", "{{ ds }}") + return context["task"].render_template(data, context, jinja_env) + + + t = BashOperator(task_id="complex_templated_echo", bash_command=build_complex_command, dag=dag) + You can use Jinja templating with every parameter that is marked as "templated" in the documentation. Template substitution occurs just before the ``pre_execute`` function of your operator is called. You can also use Jinja templating with nested fields, as long as these nested fields are marked as templated in the structure they belong to: fields registered in ``template_fields`` property will be submitted to template substitution, like the ``path`` field in the example below: From 12ad6ad1d565b39650f4209210e499d11ff3a120 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Thu, 6 Jun 2024 11:03:19 +0800 Subject: [PATCH 09/11] Touch up documentation --- .../core-concepts/operators.rst | 107 ++++++++++-------- 1 file changed, 57 insertions(+), 50 deletions(-) diff --git a/docs/apache-airflow/core-concepts/operators.rst b/docs/apache-airflow/core-concepts/operators.rst index 2e8db0099092c..176fe0cdb8204 100644 --- a/docs/apache-airflow/core-concepts/operators.rst +++ b/docs/apache-airflow/core-concepts/operators.rst @@ -86,27 +86,33 @@ For example, say you want to pass the start of the data interval as an environme Here, ``{{ ds }}`` is a templated variable, and because the ``env`` parameter of the ``BashOperator`` is templated with Jinja, the data interval's start date will be available as an environment variable named ``DATA_INTERVAL_START`` in your Bash script. -Same for the parameter ``bash_command`` that is also templated with Jinja. +You can also pass in a callable instead when Python is more readable than a Jinja template. The callable must accept two named arguments ``context`` and ``jinja_env``: + .. code-block:: python - t = BashOperator(task_id="simple_templated_echo", bash_command="echo {{ ds }}", dag=dag) + def build_complex_command(context, jinja_env): + with open("file.csv") as f: + return do_complex_things(f) -You can also perform templating manually by giving a callable when the value is more complex and/or dynamic -.. code-block:: python + t = BashOperator( + task_id="complex_templated_echo", + bash_command=build_complex_command, + dag=dag, + ) - def build_complex_command(context, jinja_env): - with open("file.csv") as f: - data = do_complex_things(f) - data.replace("THE_DATE", "{{ ds }}") - return context["task"].render_template(data, context, jinja_env) +Since each template field is only rendered once, the callable's return value will not go through rendering again. Therefore, the callable must manually render any templates. This can be done by calling ``render_template()`` on the current task like this: +.. code-block:: python - t = BashOperator(task_id="complex_templated_echo", bash_command=build_complex_command, dag=dag) + def build_complex_command(context, jinja_env): + with open("file.csv") as f: + data = do_complex_things(f) + return context["task"].render_template(data, context, jinja_env) -You can use Jinja templating with every parameter that is marked as "templated" in the documentation. Template substitution occurs just before the ``pre_execute`` function of your operator is called. +You can use templating with every parameter that is marked as "templated" in the documentation. Template substitution occurs just before the ``pre_execute`` function of your operator is called. -You can also use Jinja templating with nested fields, as long as these nested fields are marked as templated in the structure they belong to: fields registered in ``template_fields`` property will be submitted to template substitution, like the ``path`` field in the example below: +You can also use templating with nested fields, as long as these nested fields are marked as templated in the structure they belong to: fields registered in ``template_fields`` property will be submitted to template substitution, like the ``path`` field in the example below: .. code-block:: python @@ -229,64 +235,65 @@ Alternatively, if you want to prevent Airflow from treating a value as a referen Rendering Fields as Native Python Objects ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -By default, all the ``template_fields`` are rendered as strings. - -Example, let's say ``extract`` task pushes a dictionary -(Example: ``{"1001": 301.27, "1002": 433.21, "1003": 502.22}``) to :ref:`XCom ` table. -Now, when the following task is run, ``order_data`` argument is passed a string, example: -``'{"1001": 301.27, "1002": 433.21, "1003": 502.22}'``. - -.. code-block:: python - - transform = PythonOperator( - task_id="transform", - op_kwargs={"order_data": "{{ti.xcom_pull('extract')}}"}, - python_callable=transform, - ) - -If you instead want the rendered template field to return a Native Python object (``dict`` in our example), -you can pass ``render_template_as_native_obj=True`` to the DAG as follows: +By default, all Jinja templates in ``template_fields`` are rendered as strings. This however not always desired. For example, let's say an ``extract`` task pushes a dictionary ``{"1001": 301.27, "1002": 433.21, "1003": 502.22}`` to :ref:`XCom `: .. code-block:: python - dag = DAG( - dag_id="example_template_as_python_object", - schedule=None, - start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, - render_template_as_native_obj=True, - ) - - @task(task_id="extract") def extract(): data_string = '{"1001": 301.27, "1002": 433.21, "1003": 502.22}' return json.loads(data_string) +If a task depends on ``extract``, ``order_data`` argument is passed a string ``"{'1001': 301.27, '1002': 433.21, '1003': 502.22}"``: + +.. code-block:: python + def transform(order_data): - print(type(order_data)) - total_order_value = 0 - for value in order_data.values(): - total_order_value += value + total_order_value = sum(order_data.values()) # Fails because order_data is a str :( return {"total_order_value": total_order_value} - extract_task = extract() + transform = PythonOperator( + task_id="transform", + op_kwargs={"order_data": "{{ ti.xcom_pull('extract') }}"}, + python_callable=transform, + ) + + extract() >> transform + +There are two solutions if we want to get the actual dict instead. The first is to use a callable: + +.. code-block:: python - transform_task = PythonOperator( + def render_transform_op_kwargs(context, jinja_env): + order_data = context["ti"].xcom_pull("extract") + return {"order_data": order_data} + + + transform = PythonOperator( task_id="transform", - op_kwargs={"order_data": "{{ti.xcom_pull('extract')}}"}, + op_kwargs=render_transform_op_kwargs, python_callable=transform, ) - extract_task >> transform_task +Alternatively, Jinja can also be instructed to render a native Python object. This is done by passing ``render_template_as_native_obj=True`` to the DAG. This makes Airflow use `NativeEnvironment `_ instead of the default ``SandboxedEnvironment``: + +.. code-block:: python -In this case, ``order_data`` argument is passed: ``{"1001": 301.27, "1002": 433.21, "1003": 502.22}``. + with DAG( + dag_id="example_template_as_python_object", + schedule=None, + start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), + catchup=False, + render_template_as_native_obj=True, + ): + transform = PythonOperator( + task_id="transform", + op_kwargs={"order_data": "{{ ti.xcom_pull('extract') }}"}, + python_callable=transform, + ) -Airflow uses Jinja's `NativeEnvironment `_ -when ``render_template_as_native_obj`` is set to ``True``. -With ``NativeEnvironment``, rendering a template produces a native Python type. .. _concepts:reserved-keywords: From 851328af5c986d5649ffaa1fefaf8289e82b6c68 Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Thu, 6 Jun 2024 09:54:34 +0200 Subject: [PATCH 10/11] review 7 --- tests/operators/test_python.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py index e419b7de196d8..45acd854eefb2 100644 --- a/tests/operators/test_python.py +++ b/tests/operators/test_python.py @@ -229,6 +229,21 @@ def test_python_callable_keyword_arguments_are_templatized(self): assert rendered_op_kwargs["a_date"] == date(2019, 1, 1) assert rendered_op_kwargs["a_templated_string"] == f"dag {self.dag_id} ran on {self.ds_templated}." + def test_python_callable_keyword_arguments_callable_not_templatized(self): + """Test PythonOperator op_kwargs are not templatized if it's a callable""" + + def a_fn(): + return 4 + + task = self.render_templates( + lambda: 0, + op_kwargs={ + "a_callable": a_fn, + }, + ) + rendered_op_kwargs = task.op_kwargs + assert rendered_op_kwargs["a_callable"] == a_fn + def test_python_operator_shallow_copy_attr(self): def not_callable(x): raise RuntimeError("Should not be triggered") From b192863d5d038fd50cb50200a526dd4969c25c9e Mon Sep 17 00:00:00 2001 From: raphaelauv Date: Tue, 11 Jun 2024 20:02:37 +0200 Subject: [PATCH 11/11] review 8 --- docs/apache-airflow/core-concepts/operators.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/core-concepts/operators.rst b/docs/apache-airflow/core-concepts/operators.rst index 176fe0cdb8204..354697c308537 100644 --- a/docs/apache-airflow/core-concepts/operators.rst +++ b/docs/apache-airflow/core-concepts/operators.rst @@ -235,7 +235,7 @@ Alternatively, if you want to prevent Airflow from treating a value as a referen Rendering Fields as Native Python Objects ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -By default, all Jinja templates in ``template_fields`` are rendered as strings. This however not always desired. For example, let's say an ``extract`` task pushes a dictionary ``{"1001": 301.27, "1002": 433.21, "1003": 502.22}`` to :ref:`XCom `: +By default, all Jinja templates in ``template_fields`` are rendered as strings. This however is not always desired. For example, let's say an ``extract`` task pushes a dictionary ``{"1001": 301.27, "1002": 433.21, "1003": 502.22}`` to :ref:`XCom `: .. code-block:: python