Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Nested templated variables do not always render #13559

Closed
smith-m opened this issue Jan 8, 2021 · 12 comments · Fixed by #18516
Closed

Nested templated variables do not always render #13559

smith-m opened this issue Jan 8, 2021 · 12 comments · Fixed by #18516
Labels
area:Scheduler Scheduler or dag parsing Issues kind:bug This is a clearly a bug

Comments

@smith-m
Copy link

smith-m commented Jan 8, 2021

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'}
@smith-m smith-m added the kind:bug This is a clearly a bug label Jan 8, 2021
@boring-cyborg
Copy link

boring-cyborg bot commented Jan 8, 2021

Thanks for opening your first issue here! Be sure to follow the issue template!

@turbaszek
Copy link
Member

This seems to be also issue in 2.0:

[2021-01-08 09:46:58,236] {logging_mixin.py:103} INFO - level 0 arg0: level_0_nested_render_2021-01-08
[2021-01-08 09:46:58,239] {logging_mixin.py:103} INFO - level 1 kwarg1: level_1_level_0_{{task.task_id}}_{{ds}}

but it is consistent with rendered values in web ui:
Screenshot 2021-01-08 at 10 51 02

@RosterIn
Copy link
Contributor

RosterIn commented Jan 8, 2021

I don't know if this is a bug or by design but there are many votes for this in https://stackoverflow.com/questions/44855949/make-custom-airflow-macros-expand-other-macros

@turbaszek
Copy link
Member

I'm afraid that this is not simple to fix. The main problem I see is that template fields are rendered in the order they are defined in op.template_fields. This means that some values may reference values that were not yet rendered and this will cause a problem.

@smith-m
Copy link
Author

smith-m commented Jan 8, 2021

One possibility here is using the previous state of each attribute and rendering each templated_field until all fields are stable.
This will not address macro expansion in the stackoverflow example, but would support nested jinja

@vikramkoka vikramkoka added the area:UI Related to UI/UX. For Frontend Developers. label Jan 14, 2021
@turbaszek turbaszek added area:Scheduler Scheduler or dag parsing Issues and removed area:UI Related to UI/UX. For Frontend Developers. labels Jan 18, 2021
@turbaszek
Copy link
Member

One possibility here is using the previous state of each attribute and rendering each templated_field until all fields are stable.

I'm wondering how this may impact performance. Maybe it would be worth to have proof of concept?

WDYT @kaxil @XD-DENG ?

@XD-DENG
Copy link
Member

XD-DENG commented Jan 18, 2021

I agree with @turbaszek on the potential complexity of "fixing" this.

What I'm thinking is: if we can have different method (other than templating) to achieve the same functionality/purpose, we don't have to bother with it. Correct me if I'm wrong please.

@utkarshgupta137
Copy link

utkarshgupta137 commented Sep 18, 2021

I was having a similar problem: I wanted to pass a templated argument to a function, but it kept getting passed as raw template instead of rendered one.

The solution is to call the macro inside a pythonoperator & pass the data using xcoms.

def get_date(**kwargs):
    # {{ dag_run.conf.get("date", "{{ ds_no_dash }}") }} doesn't work

    ds = kwargs["ds_no_dash"]
    date = kwargs["dag_run"].conf.get("date", ds)
    kwargs["ti"].xcom_push("date", base_data)

@uranusjr
Copy link
Member

If we support nested rendering we’d also need to worry about circular reference, which would require a significantly more elaborated algorithm to properly handle. Dependency resolution is hell, don’t go there if at all possible.

@potiuk
Copy link
Member

potiuk commented Sep 25, 2021

I looked at this and the problem turned out to be a bit different.

True - we should not even attempt to solve the recursive rendering of fields cross-referencing each other. But this was not the case. This was a kwargs field which referred opargs field - and kwargs was AFTER opargs in the list of templated fields, so it should work in principle, without resolving the recursive problem.

I looked at it and it turned out, this was a problem introduced by #8805 - where instead of the original task we started to use a copy of the task locked for execution. Unfortunately - the task in context was still the original one, not the copy, which caused the rendering problem (as rendered fields in the 'context' task were not updated at all.

This means that if someone would use the context['task'] later in Python Callable or custom operator, the fields were not rendered there either 😱 .

I fixed that in the #18516 by replacing the task in context with the same locked-down copy - not sure if this might have other side effects (but I think it could only be positive side effects :D ).

@potiuk
Copy link
Member

potiuk commented Sep 25, 2021

With the change from #18516, I got this:

Screenshot 2021-09-25 12 16 03

potiuk added a commit to potiuk/airflow that referenced this issue Sep 27, 2021
When we are referring ``{{task}}` in jinja templates we can also
refer some of the fields, which are templated. We are not
able to solve all the problems with such rendering (specifically
recursive rendering of the fields used in JINJA templating might
be problematic. Currently whether you see original, or rendered
field depends solely on the sequence in templated_fields.

However that would not even explain the rendering problem
described in apache#13559 where kwargs were defined after opargs and
the rendering of opargs **should** work. It turned out that
the problem was with a change introduced in apache#8805 which made
the context effectively holds a DIFFERENT task than the current
one. Context held an original task, and the curren task was
actually a locked copy of it (to allow resolving upstream
args before locking). As a result, any changes done by
rendering templates were not visible in the task accessed
via {{ task }} jinja variable.

This change replaces the the task stored in context with the
same copy that is then used later during execution so that
at least the "sequential" rendering works and templated
fields which are 'earlier' in the list of templated fields
can be used (and render correctly) in the following fields.

Fixes: apache#13559
potiuk added a commit that referenced this issue Sep 28, 2021
When we are referring ``{{task}}` in jinja templates we can also
refer some of the fields, which are templated. We are not
able to solve all the problems with such rendering (specifically
recursive rendering of the fields used in JINJA templating might
be problematic. Currently whether you see original, or rendered
field depends solely on the sequence in templated_fields.

However that would not even explain the rendering problem
described in #13559 where kwargs were defined after opargs and
the rendering of opargs **should** work. It turned out that
the problem was with a change introduced in #8805 which made
the context effectively holds a DIFFERENT task than the current
one. Context held an original task, and the curren task was
actually a locked copy of it (to allow resolving upstream
args before locking). As a result, any changes done by
rendering templates were not visible in the task accessed
via {{ task }} jinja variable.

This change replaces the the task stored in context with the
same copy that is then used later during execution so that
at least the "sequential" rendering works and templated
fields which are 'earlier' in the list of templated fields
can be used (and render correctly) in the following fields.

Fixes: #13559
@rkranjancs1014
Copy link

"{{var.value.get("const_"+dag_run.conf.get("arg1", ""))}}"

this works for me. here I am getting arg1 one as argument in dag run and creating variable at run time then accessing variable from var.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:Scheduler Scheduler or dag parsing Issues kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

9 participants