Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Fix plugin macros not being exposed through airflow.macros #12788

Merged
merged 9 commits into from
Dec 7, 2020

Conversation

RikHeijdens
Copy link
Contributor

This PR fixes an issue where macros that are being provided through plugins can not be used at template time because they are not accessible through the airflow.macros module.

This PR consists out of two commits. In the first commit I add a test-case that reproduces the issue as outlined in #12785, and the second commit introduces the fix which fixes the issue and allows the test-case to pass.

Fixes: #12785


^ Add meaningful description above

Read the Pull Request Guidelines for more information.
In case of fundamental code change, Airflow Improvement Proposal (AIP) is needed.
In case of a new dependency, check compliance with the ASF 3rd Party License Policy.
In case of backwards incompatible changes please leave a note in UPDATING.md.

@boring-cyborg
Copy link

boring-cyborg bot commented Dec 3, 2020

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 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 Consider adding an example DAG that shows how users should use it.
  • Consider using Breeze environment 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 for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
  • Be sure to read the Airflow Coding style.
    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

# Verify that the symbol table in airflow.macros has been updated with an entry for
# this plugin, this is necessary in order to allow the plugin's macros to be used when
# rendering templates.
assert hasattr(macros, MacroPlugin.name)
Copy link
Contributor Author

@RikHeijdens RikHeijdens Dec 3, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

integrate_macros_plugins() has the side-effect that it modifies the airflow.macros module. Do we need to add additional cleanup logic to reset the contents of airflow.macros through a finalizer here?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will importlib.reload help?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Simply reloading the module using importlib.reload() will not be sufficient as the module's symbol table (dictionary) is retained.

When a module is reloaded, its dictionary (containing the module’s global variables) is retained. Redefinitions of names will override the old definitions, so this is generally not a problem. If the new version of a module does not define a name that was defined by the old version, the old definition remains.

I think we'll have to delete the module from sys.modules and import it again in order to properly remove the entries that are being added by this test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've implemented this in c2b5354 -- I'm curious to your thoughts.

Copy link
Member

@mik-laj mik-laj Dec 3, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I love this change.😻 I'm just wondering if it's worth moving this code to the context manager mock_plugins_manager. This allows us to limit the side effects in other tests as well. What do you think?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I considered doing this, but the __doc__ comment on the mock_plugin_manager seems to suggest that the scope of that mock is limited to the airflow.plugins module.

While that mock does in fact clear out the macros_modules variable in airflow.plugins, it does not actually attempt to reverse any (side) effects that are caused by invoking integrate_macros_plugins(). I did a bit of searching through the code base, and it doesn't really appear that there is any test coverage for this function beyond the test I've just added.

Because I want to avoid scope creep for the mock_plugin_manager fixture, and this is the only test case that actually appears to have to deal with side effects cause by calling integrate_macros_plugins(), I'm a bit hesitant to make changes beyond what's being proposed here.

airflow/plugins_manager.py Outdated Show resolved Hide resolved
@github-actions github-actions bot added the full tests needed We need to run full set of tests for this PR to merge label Dec 3, 2020
@github-actions
Copy link

github-actions bot commented Dec 3, 2020

The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest master at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.

@mik-laj
Copy link
Member

mik-laj commented Dec 3, 2020

@RikHeijdens Can you do a rebase? I want to make sure that importlib tricks work on all versions of Python.

@RikHeijdens
Copy link
Contributor Author

@mik-laj I've just rebased my branch on master.

@RikHeijdens
Copy link
Contributor Author

RikHeijdens commented Dec 4, 2020

@mik-laj I noticed the change proposed here causes the TestPythonVirtualenvOperator.test_airflow_context test to fail. I believe this is happening because airflow.macros is being serialized as part of the context which is passed to the underlying subprocess.

The subprocess in which the Python callable runs however does not integrate the macros provided by plugins. I was able to get this test to pass locally by adding the following two lines of code to airflow/utils/python_virtualenv_script.jinja2 (see RikHeijdens@b2c035c):

# Ensure airflow.macros has been loaded properly.
import airflow.plugins_manager
airflow.plugins_manager.integrate_macros_plugins()

Doing this however, seems like a bit of a nasty solution to me and also causes additional test failures. I'm curious whether you have any thoughts on what would be a better solution? What is the use-case of allowing subprocess spawned by PythonVirtualenvOperator to access plugin-provided macros?

Edit: I went ahead and removed airflow.macros from the set of AIRFLOW_SERIALIZABLE_CONTEXT_KEYS in PythonVirtualEnvironmentOperator. I'm not sure whether this can be considered as a backwards-incompatible change given that this functionality is currently broken, but hopefully this should allow all tests to pass at the very least.

@mik-laj
Copy link
Member

mik-laj commented Dec 4, 2020

Doing this however, seems like a bit of a nasty solution to me and also causes additional test failures

I guess this code should only be called if you have airflow installed in your environment.

def _is_airflow_env():
return self.system_site_packages or 'apache-airflow' in self.requirements

The use case is that we want to allow the user to perform any operation, but in a virtual environment. The virtual environment will allow they to use additional libraries.

@RikHeijdens
Copy link
Contributor Author

@mik-laj Got it. I've restored access to airflow.macros for subprocesses. However, in order to get the tests to pass I did have to implement an additional safe guard that would prevent airflow.plugins_manager.integrate_macros_plugins from being invoked on virtual environments that run Python 2. I don't see how this feature can be supported for any Py2 virtual environments any longer.

@mik-laj
Copy link
Member

mik-laj commented Dec 7, 2020

@RikHeijdens Got it. Thanks for your work.

@ashb ashb added this to the Airflow 2.0.0rc1 milestone Dec 7, 2020
@ashb
Copy link
Member

ashb commented Dec 7, 2020

@RikHeijdens Can you document this py2 virtualenv limiation somewhere please? (VirtualEnvOperator docs?) LGTM after that.

@RikHeijdens
Copy link
Contributor Author

I've documented the limitation in d17e98f, but I'm a bit skeptical as to whether any of the AIRFLOW_SERIALIZABLE_CONTEXT_KEYS can be used in a virtual environment that runs Python 2.

I believe anything that will require to import airflow.* no longer works, but there is also no test-coverage that confirms this. I also want to prevent any additional scope-creep with this PR, so I think it may be better to make a new issue for this.

@potiuk potiuk added the priority:critical Showstopper bug that should be patched immediately label Dec 7, 2020
@github-actions
Copy link

github-actions bot commented Dec 7, 2020

The Workflow run is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.

@RikHeijdens
Copy link
Contributor Author

@mik-laj It looks like the CI Build for d17e98f failed due to an issue that appears to be unrelated to my proposed changes. Should I rebase my commits on master again in order to trigger a new build?

@mik-laj
Copy link
Member

mik-laj commented Dec 7, 2020

@RikHeijdens Yes. Please, do it. That should fix the problem.

Rik Heijdens added 4 commits December 7, 2020 17:02
Added a test case to reproduce the issue reported in
apache#12785
In order to allow a plugin-provided macro to be used at templating time,
it needs to be exposed through the airflow.macros module.

This commit fixes GitHub issue airflow#12785.
This test-case has side-effects in the sense that the symbol table of
the airflow.macros module is altered when integrate_macros_plugins() is
invoked. This commit adds a finalizer to the test case that ensures that
that module is being reloaded completely in order to prevent impact on
other tests.
Rik Heijdens added 5 commits December 7, 2020 17:02
The process that runs in the virtual environment may not be able to
access macros that have been provided by plugins. E.g. in cases where
the virtual environment uses a different major version of Python than
Airflow itself.

In order to prevent this from being an issue macros has been removed
from the serializable context.
This reverts commit 5a6c3cfd19c584a2b5ab2220ee580f8fd85ddb17.
When Airflow is available in a virtual environment, and when this
environment runs at least Python 3, then plugin-provided macros should
be made available to the Python callable that is being executed in this
environment.
Plugin-provided macros can not be used on Python 2 when using
PythonVirtualenvOperator any longer.
@ashb
Copy link
Member

ashb commented Dec 7, 2020

Almost there. It almost feels like someone is running a slowloris attack against GitHub actions today.

@ashb
Copy link
Member

ashb commented Dec 7, 2020

✔️

@ashb ashb merged commit 29d7848 into apache:master Dec 7, 2020
@boring-cyborg
Copy link

boring-cyborg bot commented Dec 7, 2020

Awesome work, congrats on your first merged pull request!

@RikHeijdens RikHeijdens deleted the fix-plugin-macros branch December 8, 2020 08:51
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:plugins full tests needed We need to run full set of tests for this PR to merge priority:critical Showstopper bug that should be patched immediately
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Macros added through plugins can not be used within Jinja templates in Airflow 2.0
5 participants