Macros added through plugins can not be used within Jinja templates in Airflow 2.0 #12785
Labels
affected_version:2.0
Issues Reported for 2.0
kind:bug
This is a clearly a bug
priority:critical
Showstopper bug that should be patched immediately
Milestone
Apache Airflow version: 2.0.0b3
Kubernetes version (if you are using kubernetes) (use
kubectl version
): N/AEnvironment:
uname -a
): Linux 6ae65b86e112 5.4.0-52-generic I need a flow to start with a sensor #57-Ubuntu SMP Thu Oct 15 10:57:00 UTC 2020 x86_64 GNU/LinuxWhat happened:
At JW Player we add additional macros to Airflow through a plugin. The definition of this plugin looks like the following (simplified):
convert_image_tag
is a function that takes a string (a docker tag) as argument and resolves it to a SHA-256 hash that uniquely identifies an image by querying the docker registry. I.e. it is a function that takes a string as argument and returns a string.In Airflow 1.10.x we can successfully use this macro in our DAGs to resolve image tags to SHA-256 hashes, e.g. the following DAG will run an Alpine Image using a DockerOperator:
This is in contrast to Airflow 2.0, if we attempt to use our custom macro here, then when Airflow attempts to render the task template it will error out with the following error:
What you expected to happen:
I would have expected that the DAG definition from above would have worked in Airflow 2.0, like it would have functioned in Airflow 1.10.x.
How to reproduce it:
This bug can be reproduced by creating a plugin that adds a macro, and then attempting to use that macro in a DAG.
Anything else we need to know:
In order to better understand the issue, I did a bit of digging. The plugin that we extend Airflow's functionality with has its own suite of pytest testcases. Since we are in the process of preparing for a transition to Airflow 2.0 we are now running the unit tests for this plugin against both Airflow 1.10.x and Airflow 2.0.0b3.
After reviewing how plugins were being loaded in Airflow, I've added the following testcase to mimic how plugins were being loaded and how
get_template_context()
in Airflow 2.0 ensures that plugins have been imported:This test case passes when being ran on Airflow 1.10, but surprisngly enough it fails on Airflow 2.x. Specifically it fails on the
assert hasattr(macros, 'jw_utils')
statement in Airflow 2.0. This statement tests whether the macros that we create through theJwUtilsPlugin
have been properly added toairflow.macros
.I thought it was strange for the test-case to fail on this module, given that the
import_module()
statement succeeded in Airflow 2.0. After this observation I started comparing the logic for registering macros in Airflow 1.10.x to the Airflow 2.0.0 implementation.While doing this I observed that the plugin loading mechanism in Airflow 1.10.x works because Airflow automatically discovers all plugins through the
plugins_manager
module. When this happens it automatically initializes plugin-macro modules in theairflow.macros
namespace. Notably, after the plugin's module has been initialized it will also automatically be registered on theairflow.macros
module by updating the dictionary returned byglobals()
.This is in contrast to Airflow 2.0, where plugins are no longer loaded automatically. Instead they are being loaded lazily, i.e. they will be loaded on-demand whenever a function needs them. In order to load macros (or ensure that macros have been loaded), modules need to import the
integrate_macros_plugins
function fromairflow.plugins_manager
.When Airflow attempts to prepare a template context, prior to running a task, it properly imports this function and invokes it in taskinstance.py. However, in contrast to the old 1.10.x implementation, this function does not update the symbol table of
airflow.macros
. The result of this is that the macros from the plugin will in fact be imported, but becauseairflow.macros
symbol table itself is not being updated, the macros that are being added by the plugins can not be used in the template rendering context.I believe this issue could be solved by ensuring that
integrate_macros_plugins
sets a reference to theairflow.macros.jw_utils
asjw_utils
on theairflow.macros
module. Once that has been done I believe macros provided through plugins are functional again.The text was updated successfully, but these errors were encountered: