Skip to content

Nested templated variables do not always render #13559

@smith-m

Description

@smith-m

Apache Airflow version:
1.10.14 and 1.10.8.

Environment:
Python 3.6 and Airflow 1.10.14 on sqllite,

What happened:

Nested jinja templates do not consistently render when running tasks. TI run rendering behavior also differs from airflow UI and airflow render cli.

What you expected to happen:

Airflow should render nested jinja templates consistently and completely across each interface. Coming from airflow 1.8.2, this used to be the case.

This regression may have been introduced in 1.10.6 with a refactor of BaseOperator templating functionality.
#5461

Whether or not a nested layer renders seems to differ based on which arg is being templated in an operator and perhaps order. Furthermore, it seems like the render cli and airflow ui each apply TI.render_templates() a second time, creating inconsistency in what nested templates get rendered.

There may be bug in the way BaseOperator.render_template() observes/caches templated fields

How to reproduce it:
From the most basic airflow setup

nested_template_bug.py

from datetime import datetime
from airflow import DAG
from airflow.operators.python_operator import PythonOperator

with DAG("nested_template_bug", start_date=datetime(2021, 1, 1)) as dag:

    arg0 = 'level_0_{{task.task_id}}_{{ds}}'
    kwarg1 = 'level_1_{{task.op_args[0]}}'

    def print_fields(arg0, kwarg1):
        print(f'level 0 arg0: {arg0}')
        print(f'level 1 kwarg1: {kwarg1}') 

    nested_render = PythonOperator(
        task_id='nested_render',
        python_callable=print_fields,
        op_args=[arg0, ],
        op_kwargs={
            'kwarg1': kwarg1,
        },
    )
> airflow test c

level 0 arg0: level_0_nested_render_2021-01-01
level 1 kwarg1: level_1_level_0_{{task.task_id}}_{{ds}}

> airflow render nested_template_bug nested_render 2021-01-01

# ----------------------------------------------------------
# property: op_args
# ----------------------------------------------------------
['level_0_nested_render_2021-01-01']

# ----------------------------------------------------------
# property: op_kwargs
# ----------------------------------------------------------
{'kwarg1': 'level_1_level_0_nested_render_2021-01-01'}

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:Schedulerincluding HA (high availability) schedulerkind:bugThis is a clearly a bug

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions