Skip to content

Commit

Permalink
Create UndefinedJinjaVariablesRule (Resolves apache#11144)
Browse files Browse the repository at this point in the history
Part of Issue apache#8765 - adding a rule to check for
undefined jinja variables when upgrading to Airflow2.0
Logic - Use a DagBag to pull all dags and iterate over
every dag. For every dag the task will be rendered using
an updated Jinja Environment using - jinja2.DebugUndefined
This will render the template leaving undefined variables
as they were. Using regex we can extract the variables
and present possible error cases when upgrading.
  • Loading branch information
ashmeet13 committed Oct 6, 2020
1 parent 0168f71 commit 240d8c0
Show file tree
Hide file tree
Showing 2 changed files with 164 additions and 0 deletions.
89 changes: 89 additions & 0 deletions airflow/upgrade/rules/undefined_jinja_varaibles.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import absolute_import

import re

import jinja2
import six

from airflow.models import DagBag, TaskInstance
from airflow.upgrade.rules.base_rule import BaseRule
from airflow.utils import timezone


class UndefinedJinjaVariablesRule(BaseRule):

title = "Jinja Template Variables cannot be undefined"

description = """\
Jinja Templates have been updated to the following rule - jinja2.StrictUndefined
With this change a task will fail if it recieves any undefined variables.
"""

def check_rendered_content(self, rendered_content):
if isinstance(rendered_content, six.string_types):
return set(re.findall(r"{{(.*?)}}", rendered_content))

elif isinstance(rendered_content, (tuple, list, set)):
parsed_templates = set()
for element in rendered_content:
parsed_templates.union(self.check_rendered_content(element))
return parsed_templates

elif isinstance(rendered_content, dict):
parsed_templates = set()
for key, value in rendered_content.items():
parsed_templates.union(self.check_rendered_content(str(value)))
return parsed_templates

def check(self, dagbag=DagBag()):
dags = dagbag.dags
messages = []
for dag_id, dag in dags.items():
bracket_pattern = r"\[(.*?)\]"
dag.template_undefined = jinja2.DebugUndefined
for task in dag.tasks:
task_instance = TaskInstance(
task=task, execution_date=timezone.utcnow()
)
template_context = task_instance.get_template_context()

rendered_content_collection = []

for attr_name in task.template_fields:
content = getattr(task, attr_name)
if content:
rendered_content_collection.append(
task.render_template(content, template_context)
)

for rendered_content in rendered_content_collection:
undefined_variables = self.check_rendered_content(rendered_content)
for undefined_variable in undefined_variables:
result = re.findall(bracket_pattern, undefined_variable)
if result:
undefined_variable = result[0].strip("'")
new_msg = (
"Possible Undefined Jinja Variable -> DAG: {}, Task: {}, "
"Variable: {}".format(
dag_id, task.task_id, undefined_variable.strip()
)
)
messages.append(new_msg)
return messages
75 changes: 75 additions & 0 deletions tests/upgrade/rules/test_undefined_jinja_varaibles.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from tempfile import mkdtemp
from unittest import TestCase

from airflow import DAG
from airflow.models import DagBag
from airflow.operators.bash_operator import BashOperator
from airflow.upgrade.rules.undefined_jinja_varaibles import UndefinedJinjaVariablesRule
from tests.models import DEFAULT_DATE


class TestConnTypeIsNotNullableRule(TestCase):
@classmethod
def setUpClass(cls):
cls.empty_dir = mkdtemp()

def setUp(self):

self.dag = DAG(dag_id="test-undefined-jinja-variables", start_date=DEFAULT_DATE)

template_command = """
{% for i in range(5) %}
echo "{{ params.defined_variable }}"
echo "{{ execution_date.today }}"
echo "{{ params.undefined_variable }}"
echo "{{ foo }}"
{% endfor %}
"""

BashOperator(
task_id="templated_string",
depends_on_past=False,
bash_command=template_command,
params={"defined_variable": "defined_value"},
dag=self.dag,
)

self.dagbag = DagBag(dag_folder=self.empty_dir, include_examples=False)
self.dagbag.dags[self.dag.dag_id] = self.dag

def test_check(self):
rule = UndefinedJinjaVariablesRule()

assert isinstance(rule.description, str)
assert isinstance(rule.title, str)

messages = rule.check(self.dagbag)

expected_messages = [
"Possible Undefined Jinja Variable -> DAG: test-undefined-jinja-variables, "
"Task: templated_string, Variable: undefined",
"Possible Undefined Jinja Variable -> DAG: test-undefined-jinja-variables, "
"Task: templated_string, Variable: foo",
]

print(messages)
assert [m for m in messages if m in expected_messages], len(messages) == len(
expected_messages
)

0 comments on commit 240d8c0

Please sign in to comment.