You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/02 16:11:42 UTC

[GitHub] [airflow] ashmeet13 opened a new pull request #11241: Create Undefined Jinja Variables Rule

ashmeet13 opened a new pull request #11241:
URL: https://github.com/apache/airflow/pull/11241


   Hi,
   
   Adding a rule to check for Undefined Jinja Variables (Issue: #11144) 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.
   
   Since I am fairly new to Airflow - please do guide if there is a better approach to implement this rule.
   Thanks! 
   
   
   Related: #8765
   Closes: #11144


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



[GitHub] [airflow] turbaszek commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728827674


   @ashmeet13 this change looks good to me, can you please squash the commits and the rebase onto current `v1-10-test`?


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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-702822745


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, pylint and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/master/docs/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/master/BREEZE.rst) for testing locally, itโ€™s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better ๐Ÿš€.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


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



[GitHub] [airflow] github-actions[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-719453840


   [The Workflow run](https://github.com/apache/airflow/actions/runs/337296464) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728835994


   Yup. Just to confirm what exactly needs to be done?
   Do I change the target branch back to `v1-10-test` and then rebase my commits with `v1-10-test`?
   
   Could you clarify @turbaszek 


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r514263552



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,144 @@
+# 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 import conf
+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

Review comment:
       ```suggestion
   The default behavior for DAG's Jinja templates has changed. Now, more restrictive validation of non-existent variables is applied - `jinja2.StrictUndefined`.




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



[GitHub] [airflow] potiuk commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-704450038


   Not sure what's going on here but there is at least one commit that should not be there from what I see (the first one). So something's going on with this (and I See some strange/intermittent errors). can you please try to rebase again on top of v1-10-test and push ? v1-10-test seems to build all right. 
   


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-716400412


   @turbaszek I have changed the base to `v1-10-stable` and rebased it accordingly.
   There are two checks that seem to be failing - 
   1. CI Build / Core:Sqlite Py3.8 (pull_request)
   2. CI Build / Quarantined:Pg9.6,Py3.6 (pull_request)
   
   What should be done to fix them?


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r509307516



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):

Review comment:
       Fixed names in 08b0f75




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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728951438


   Yup.
   Figured it out. Just saw #development - A PR from `v1-10-test` was merged. 
   Rebased. Hopefully this works. My bad for the confusion!


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-705387876


   Hey @potiuk - took a while but I believe the branch is fixed now.
   Also the PR did get closed in the middle when trying to fix the branch. My bad on screwing up the rebase and fixing the branch.
   
   Marking the branch `ready for review`


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508489716



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):

Review comment:
       Fixing this with a better name.




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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r514925953



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,144 @@
+# 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 import conf
+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

Review comment:
       Got it. Added these instructions in `description`




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



[GitHub] [airflow] github-actions[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-705367339


   [The Build Workflow run](https://github.com/apache/airflow/actions/runs/294924033) is cancelling this PR. It in earlier duplicate of 1029499 run.


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r515933198



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Any progress?




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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r514259992



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       DAGs in which the user has decided about a specific behavior should not be checked. 
   
   ```python
   with DAG(dag_id="case_a"):
   ```
   ```python
   with DAG(dag_id="case_b", template_undefined=jinja2.StrictUndefined):
   ```
   ```python
   with DAG(dag_id="case_c", template_undefined=jinja2.DebugUndefined):
   ```
   ```python
   with DAG(dag_id="case_d", template_undefined=jinja2.Undefined):
   ```
   Only `dag_a` should be checked. The others should be omitted as they will work properly in Airlfow 2.0.
   




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



[GitHub] [airflow] ashmeet13 closed pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 closed pull request #11241:
URL: https://github.com/apache/airflow/pull/11241


   


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r516745354



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       It seems to me that it is only possible if we make changes to the DAG constructor and get_template_env method.
   https://github.com/apache/airflow/blob/d8aa24b729de4e80cd2d71f99d04cbabf5f3b36c/airflow/models/dag.py#L238
   ```diff
   -        template_undefined: Type[jinja2.StrictUndefined] = jinja2.StrictUndefined,
   +        template_undefined: Optional[Type[jinja2.StrictUndefined]] = None,
   ```
   https://github.com/apache/airflow/blob/d8aa24b729de4e80cd2d71f99d04cbabf5f3b36c/airflow/models/dag.py#L968
   ```diff
   -            'undefined': self.template_undefined,
   +            'undefined': self.template_undefined or jinja2.StrictUndefined,,
   
   ```
   After making the above changes, we can check the class attribute to test our condition without any major problems.
   




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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508490878



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+

Review comment:
       My bad. Got neglected from my end. Working on adding this logic. 




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



[GitHub] [airflow] potiuk commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728942309


   I guess that was a bad rebase :) @ashmeet13 ? Seems that it is going to be better now. 


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



[GitHub] [airflow] ashmeet13 removed a comment on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 removed a comment on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728909467


   @turbaszek I did squash my commits and I had already rebased my branch with `v1-10-stable`
   There are a few tests that seem to be failing - 
   1. CI Build / Status of image build PROD: 3.6 (pull_request)
   ```
   Error: Process completed with exit code 1.
   ```
   2. CI Build / Cancel previous workflow run (pull_request)
   ```
   Error: Unable to process command '::set-env name=WORKFLOW_ID::1029499' successfully.
   Error: The `set-env` command is disabled. Please upgrade to using Environment Files or opt into unsecure command execution by setting the `ACTIONS_ALLOW_UNSECURE_COMMANDS` environment variable to `true`. For more information see: https://github.blog/changelog/2020-10-01-github-actions-deprecating-set-env-and-add-path-commands/
   ```
   
   What should I be doing to fix these?


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



[GitHub] [airflow] potiuk commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-703164588






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



[GitHub] [airflow] ashmeet13 edited a comment on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 edited a comment on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-704726106


   I think I have messed up the branch. 
   I see that this commit - `Conditional MySQL Client installation` was included but should not have been here. 
   
   When I try to rebase again as you suggested it continues to `Applying: Conditional MySQL Client installation` which is a commit I do not know how I have managed to make here.
   
   Not really sure what to do. Sorry for the trouble. What would your suggestion be to do?


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-703390763


   Got it @potiuk. Thanks for letting me now. 


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



[GitHub] [airflow] kaxil commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-718747720


   > @turbaszek I have changed the base to `v1-10-stable` and rebased it accordingly.
   > There are two checks that seem to be failing -
   > 
   > 1. CI Build / Core:Sqlite Py3.8 (pull_request)
   > 2. CI Build / Quarantined:Pg9.6,Py3.6 (pull_request)
   > 
   > What should be done to fix them?
   
   Ignore the Quarantined one, core:Sqlite one seems to be just flaky so that can be ignored too


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-715412920


   Closed again by mistake while performing a rebase.
   My bad again!


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r514925739



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,144 @@
+# 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 import conf
+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

Review comment:
       Added this suggestion




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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-730297236


   Awesome work, congrats on your first merged pull request!
   


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r516873328



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Why does this require new PR? I think you can update this PR.




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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r517095762



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Got it. Working on the changes.




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



[GitHub] [airflow] potiuk commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-703227286


   The quarantined tests are kind and f ok to fail. Those are flaky tests that we know are not stable. The k8s ones are likely a transient error . So do not worry about those 


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r516008427



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       I missed this comment - my fault.
   How exactly would we check this?
   
   My thought process - 
   This would require the check to read the Python scripts in which the DAGs have been defined. Then read line by line where the DAG is defined and then check whether `template_undefined` is defined or not.

##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       A bit more thinking on this. We could - 
   1. Regex search the python files for a pattern where `template_undefined=jinja2.` exists.
   2. If a match occurs - search `dag_id="` using regex around the match and get the dag_id
   3. The gathered dag_id's then can be skipped for a check. 
   
   This approach does omit cases such as - 
   ```
   dag_name = "example_dag"
   example = DAG(dag_id=dag_name, template_undefined=jinja2.Undefined)
   ```
   
   
   How would you suggest writing this?  




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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-724613608


   @turbaszek requesting for a review whenever you get the chance.
   Thanks!


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728973941


   The checks have passed. Anything else needed to be done @turbaszek ?


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508008445



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+

Review comment:
       What is the expected behavior in the "else" cases?

##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+

Review comment:
       What is the expected behavior in the "else" case?




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



[GitHub] [airflow] turbaszek commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-716125287


   @ashmeet13 would you mind rebasing and changing the target base due to https://github.com/apache/airflow/pull/11719#issuecomment-714258732


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-704726106


   I think I have messed up the branch. 
   I see that this commit - `Conditional MySQL Client installation` was included but should not have been here. 
   
   When I try to rebase again as you suggested it continues to `Applying: Conditional MySQL Client installation` which is a commit I do not know how I have managed to make here.
   
   I think I'll ask for help on `airflow-how-to-pr`


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



[GitHub] [airflow] github-actions[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-715440899


   [The Workflow run](https://github.com/apache/airflow/actions/runs/324432617) is cancelling this PR. It in earlier duplicate of 1029499 run.


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



[GitHub] [airflow] ashmeet13 closed pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 closed pull request #11241:
URL: https://github.com/apache/airflow/pull/11241


   


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



[GitHub] [airflow] ashmeet13 closed pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 closed pull request #11241:
URL: https://github.com/apache/airflow/pull/11241


   


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-730296094


   Can this PR be merged @turbaszek @potiuk if no other changes required?


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508003716



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):

Review comment:
       I don't understand the name of this method. Could you use a more descriptive name?




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



[GitHub] [airflow] mik-laj commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-722026261


   @turbaszek Can you look at it?


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r509307242



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+

Review comment:
       Added the logic in 08b0f75




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



[GitHub] [airflow] github-actions[bot] commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-705367339


   [The Build Workflow run](https://github.com/apache/airflow/actions/runs/294924033) is cancelling this PR. It in earlier duplicate of 1029499 run.


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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508490487



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Didn't exactly get this?
   Which DAGs could be omitted from checking?




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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r508004175



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Do you think it is worth omitting checking some DAGs in some cases?




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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r517242128



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       Added the changes w.r.t to `v1-10-stable` as the target branch.
   The code changes you had suggested were w.r.t to `master` branch.
   
   Also added the logic to skip check for DAG when `template_undefined` is explicitly mentioned




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



[GitHub] [airflow] turbaszek commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728843450


   > Yup. Just to confirm what exactly needs to be done?
   > Do I change the target branch back to `v1-10-test` and then rebase my commits with `v1-10-test`?
   > 
   > Could you clarify @turbaszek
   
   Sorry my bad! I meant `v1-10-stable` as it is now. The was fixed so it would be nice to:
   - squash all your commits form this branch (squashing will help with rebasing)
   - rebase it onto `v1-10-stable` branch
   
   In this way we will be able to run whole CI tests hopefully 


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-703225049


   Hey @potiuk rebased the branch.
   Two of the checks seemed to have failed - 
   1. `CI Build / Quarantined tests`
   2. `CI Build / K8s: 3.6 v1.17.5 image`
   
   Could you please guide towards how should I be fixing them?


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



[GitHub] [airflow] turbaszek merged pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
turbaszek merged pull request #11241:
URL: https://github.com/apache/airflow/pull/11241


   


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



[GitHub] [airflow] mik-laj commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r514262043



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,144 @@
+# 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 import conf
+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

Review comment:
       Here it should be written that the user should correct the templates or pass ``template_undefined=jinja2.Undefined`` to the DAG.




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



[GitHub] [airflow] ashmeet13 commented on a change in pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on a change in pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#discussion_r516870925



##########
File path: airflow/upgrade/rules/undefined_jinja_varaibles.py
##########
@@ -0,0 +1,121 @@
+# 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 import conf
+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):
+        """Replicates the logic in BaseOperator.render_template() to
+        cover all the cases needed to be checked.
+        """
+        if isinstance(rendered_content, six.string_types):
+            return set(re.findall(r"{{(.*?)}}", rendered_content))
+
+        elif isinstance(rendered_content, (tuple, list, set)):
+            debug_error_messages = set()
+            for element in rendered_content:
+                debug_error_messages.union(self._check_rendered_content(element))
+            return debug_error_messages
+
+        elif isinstance(rendered_content, dict):
+            debug_error_messages = set()
+            for key, value in rendered_content.items():
+                debug_error_messages.union(self._check_rendered_content(str(value)))
+            return debug_error_messages
+
+    def _render_task_content(self, task, content, context):
+        completed_rendering = False
+        errors_while_rendering = []
+        while not completed_rendering:
+            # Catch errors such as {{ object.element }} where
+            # object is not defined
+            try:
+                renderend_content = task.render_template(content, context)
+                completed_rendering = True
+            except Exception as e:
+                undefined_variable = re.sub(" is undefined", "", str(e))
+                undefined_variable = re.sub("'", "", undefined_variable)
+                context[undefined_variable] = dict()
+                message = "Could not find the object '{}'".format(undefined_variable)
+                errors_while_rendering.append(message)
+        return renderend_content, errors_while_rendering
+
+    def _task_level_(self, task):
+        messages = {}
+        task_instance = TaskInstance(task=task, execution_date=timezone.utcnow())
+        context = task_instance.get_template_context()
+        for attr_name in task.template_fields:
+            content = getattr(task, attr_name)
+            if content:
+                rendered_content, errors_while_rendering = self._render_task_content(
+                    task, content, context
+                )
+                debug_error_messages = list(
+                    self._check_rendered_content(rendered_content)
+                )
+                messages[attr_name] = errors_while_rendering + debug_error_messages
+
+        return messages
+
+    def _dag_level_(self, dag):
+        dag.template_undefined = jinja2.DebugUndefined
+        tasks = dag.tasks
+        messages = {}
+        for task in tasks:
+            error_messages = self._task_level_(task)
+            messages[task.task_id] = error_messages
+        return messages
+
+    def check(self, dagbag=None):
+        if not dagbag:
+            dag_folder = conf.get("core", "dags_folder")
+            dagbag = DagBag(dag_folder)
+        dags = dagbag.dags
+        messages = []
+        for dag_id, dag in dags.items():
+            dag_messages = self._dag_level_(dag)

Review comment:
       I see. Makes sense.
   I believe this would require a new PR for the change?
   
   Should I raise one for the same?




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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-705387876


   Hey @potiuk - took a while but I believe the branch is fixed now.
   Also the PR did get closed in the middle when trying to fix the branch. My bad on screwing up the rebase and fixing the branch.
   
   Marking the branch `ready for review`


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



[GitHub] [airflow] ashmeet13 commented on pull request #11241: Create Undefined Jinja Variables Rule

Posted by GitBox <gi...@apache.org>.
ashmeet13 commented on pull request #11241:
URL: https://github.com/apache/airflow/pull/11241#issuecomment-728909467


   @turbaszek I did squash my commits and I had already rebased my branch with `v1-10-stable`
   There are a few tests that seem to be failing - 
   1. CI Build / Status of image build PROD: 3.6 (pull_request)
   ```
   Error: Process completed with exit code 1.
   ```
   2. CI Build / Cancel previous workflow run (pull_request)
   ```
   Error: Unable to process command '::set-env name=WORKFLOW_ID::1029499' successfully.
   Error: The `set-env` command is disabled. Please upgrade to using Environment Files or opt into unsecure command execution by setting the `ACTIONS_ALLOW_UNSECURE_COMMANDS` environment variable to `true`. For more information see: https://github.blog/changelog/2020-10-01-github-actions-deprecating-set-env-and-add-path-commands/
   ```
   
   What should I be doing to fix these?


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