Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Revise migration according to community feedback

Status

Page properties


StateVote
Discussion Threadhttps://lists.apache.org/thread/yjcgb6fhn365n3307blq4y4v50gjynsy
Vote Threadhttps://lists.apache.org/thread/s5g0zvjjxlpgnp718vs6n86qtlgzcgpw
Vote Result Thread
Progress Tracking (PR/GitHub Project/Issue Label)
Date Created

2024-07-19

Version Released
AuthorsTzu-ping Chung 



Abstract

A new way to provide templates to an Airflow operator is proposed to replace the existing implicit template detection based on templates_ext and template_fields. A base class Template is implemented to provide field-templating features, with two concrete subclasses FileTemplate and StringTemplate being planned for the initial release.

Motivation

Field templating in Airflow refers to how certain fields in Airflow operator classes can be passed in a string using the Jinja template syntax, and they are rendered at runtime to actual values used for the operator’s execution.

This is a nice feature, but the implicitness introduces a few inherent issues. Since the template detection and rendering happens automatically, it is not easy to tell what fields are template-able and what are not; while each operator does always have a template_fields that canonically indicates this, the knowledge is generally “tribal” and not very discoverable. It is also not easy to debug things when the behaviour is not expected, especially when templates_ext is involved—the TemplateNotFound issue is a common pitfall that troubles newcomers to no end.

This is probably the single most awkward pattern in Airflow 2:

# XXX: Airflow 2 syntax.
cat_file = BashOperator(
    task_id="cat_file",
    bash_command="cat {{ ds }}.sh",  # TemplateNotFound: cat {{ ds }}.sh
)

The string having a “.sh” suffix triggers templates_ext logic, making Airflow interpret the string as a path (that points to a template) and try to load it. The traditional workaround is to add a trailing space to the command:

# XXX: Airflow 2 syntax.
cat_file = BashOperator(
    task_id="cat_file",
    bash_command="cat {{ ds }}.sh ",  # Can you spot the difference?
)

This is so problematic we have a paragraph in documentation dedicated to it. Airflow 2.8 added the literal helper to improve ergonomics on this a bit, but you still need to first learn about (or more likely, see your task fail with the weird error) to remember to import.

I want to make templating explicit in Airflow 3, trading some ease of writing for better DAG legibility for less experienced Airflow users, improved debuggability when things go wrong, and providing better discoverability of whether a field is template-able.

Proposal

Instead of relying on template_fields and templates_ext to control templating behaviour, all operator fields in Airflow 3 will be non-templated by default. For a field to be template-able, it must have an argument in the operator’s __init__ matching the field name; the argument must explicitly accept a Template instance. For example:

class BashOperator(BaseOperator):
    def __init__(
        task_id: str,  # Not templatable.
        bash_command: str | Template,  # Templatable!
        env: dict[str, str] | Template | None = None,  # Templatable!
        append_env: bool = False,  # Not templatable.
        ...,
    ) -> None:
        super().__init__(task_id=task_id, ...)
        self.bash_command = bash_command
        self.env = env
        ...

It may be possible to simplify this with either dataclasses or attrs. I’m not sure if it is possible, but we will look into this during implementation. We’ll use one of the libraries if so.

The template_fields attribute will be made private and renamed to _template_fields. This is due to technical limitations in getting type annotations in old Python versions—the goal is to eventually remove it and rely on inspect.get_annotations to inspect what fields support templating instead. A pre-commit hook will be created to ensure the information in _template_fields is in sync in the meantime.

The templates_ext attribute is no longer needed.

Both template_fields and templates_ext will be kept available for compatibility considerations, but either declaring them in a subclass or accessing them will cause a deprecation warning will be removed from BaseOperator, and user code relying on the attribute to be on the operator will break. See Compatibility and Migration below for migration suggestions.

Template sources

Two subclasses of Template will be introduced to Airflow 3.0:

  • StringTemplate takes a string argument. The string should use the Jinja2 template syntax, and is rendered at runtime.
  • FileTemplate takes an argument that points to a file. The file’s content is loaded as the template string and rendered at runtime.

The Template base class is considered a public API, and users are free to subclass from it. Custom subclasses are resolved like built-in ones, assuming the implementation is correct. Documentation on correctly implementing subclasses will be provided. Later Airflow versions may also introduce more subclasses.

With explicit template declaration, the pattern mentioned in the Motivation section becomes:

cat_file = BashOperator(
    task_id="cat_file",
    bash_command=StringTemplate("cat {{ ds }}.sh"),
)

Airflow knows exactly this is supposed to be a template string, not a path to the template (for which you’d use FileTemplate instead), and does the correct thing.

This proposal does not aim to change how the templates are resolved and rendered at runtime. If a path passed to FileTemplate is relative, for example, resolution is still controlled by the DAG’s template_searchpath. Macros and filters also stay unchanged.

Rendering a non-string value

A new argument converter will be added to Template. If supplied, this should be a callable that takes exactly one argument—the rendered result—and converts it to a new value. The general use case is to render a field to a non-string type, such as:

StringTemplate("{{ ds }}", converter=datetime.date.fromisoformat)

The above is rendered into the date specified by ds in the Python date type.

Another interesting use case is to use ast.literal_eval as the converter to create a list, dict, or other richer Python structures. This should be a nice alternative to the somewhat clunky and restrictive render_template_as_native_obj. We can accept any callable here, including user-defined ones, without going through the plugin architecture, because the code will only be called in the worker.

Per-field arguments

Requiring all templating values to be wrapped in a class also introduces the possibility of per-field configurations. For example, you will be able to do

print_foo = BashOperator(
    task_id="print_foo",
   bash_command=StringTemplate(r"printf '$foo\n$bar'"),
    env=FileTemplate(
       "cat_file_env.json",
        template_searchpath=["/etc/my_templates/"],
        converter=json.loads,
    ),
)

Then, you can load the template file at /etc/my_templates/cat_file_env.json with content

{"foo": "123", "bar": "456"}

Nesting

Another interesting way to use the classes is to have nesting templates, which was previously impossible:

run_some_file = BashOperator(
    task_id="run_some_file",
    bash_command=FileTemplate(StringTemplate("{{ ds }}.sh")),
)

This loads a file at runtime corresponding to ds, and renders the content as a template. The example here is artificially contrived, but a similar pattern has been raised sporadically as a feature request. It is possible to work around the lack of this feature, but the syntax allows users more expressiveness when writing DAGs.

Compatibility and Migration

This is obviously a big Airflow 3.0 is a major release, and it is technically acceptable to make this a breaking change and will require users to rewrite a lot of DAGs. The main way to approach this would be with a migration tool to automatically rewrite DAG.

One thing I explicitly do not want to do is to keep a flag in Airflow 3 (either globally or otherwise) for users to switch back to the previous behaviour. This flag would require Airflow 3 to keep the template_fields and templates_ext attributes practically forever, which would be dead code if we expected people to use the new syntax.

require all user DAGs to be rewritten to accomodate this change. However, the Airflow community feels strongly that it is essential to keep compatibility in this case. Various measures are therefore put into place to enable more users a smoother migration.

Most significantly, we are going to keep the Alrfow 2 syntax working in Airflow 3. This requires compatibility code to be added in template-rendering functions (e.g. render_template) so we can correctly handle nested templates (template strings in containers e.g. dict or list). This compatibility logic will be behind a configuration flag and disabled by default. This ensures new Airflow installations are on the new syntax when possible, but also minimise work for existing users to migrate an Airflow 2 installation.

A migration tool will also be provided to automatically rewrite DAG, in the lines of ruff and other linter-fixer tools.

One other significant The approach I would like to take instead is to have a compatibility layer on core Airflow to handle migration. This takes inspiration from Python’s six package, providing an interface for users to write code that’s both compatible to both old and modern syntaxes, and get the same behaviour at runtime. This could be fulfilled by either airflow.providers.compat, or a separate package entirely—the latter option would allow us to easier evaluate when to stop developing the compatibility layer, potentially reducing maintenance overhead caused by mixing different compatibility concerns (toward other providers, and toward end users) in one provider. We can decide later either way, as late as Airflow 3’s release.

Make an Airflow 2 operator compatible to 3

We should also provider provide a utility to simply operators’ migration, used like this:

from airflow.operators.base import BaseOperator
from airflow.templates import StringTemplate
from airflow_compat import new_template_interface

@new_template_interface
class MyLongHistoryOperator(BaseOperator):
# Operaqtor implemented to use the old style.
template_fields = ...
templates_ext = ...
my_task = MyUnpopularOperator(
   task_id="my_task",
 value=StringTemplate("{{ ti.run_id }}"), # But now takes the new syntax!
)

When running on Airflow 2, the decorator would automatically provide a new __init__ method for the oprator that goes through template_fields and automatically convert template objects back to Airflow 2-style template strings when the DAG is parsed. Airflow 2-style implicit template strings continue to work on Airflow 2 when this decorator is added.

The template classes will be back-ported into the compatibility layer, so users migrate to the new syntax without needing to upgrade first. Per-field arguments are rejected both on Airflow 2 and 3 by the decorated operator since it is impossible to make them work on an operator designed to work the old syntax.

Since this is a decorator, users may also manually “fix” a less-maintained operator in many cases:

from third_party_package.operators import MyLongHistoryOperator as Airflow2MyLongHistoryOperator

MyLongHistoryOperator = new_template_interface(Airflow2MyLongHistoryOperator)

This is intended as a last-ditch solution when a library is abandoned. We’ll also add checks in the decorator to make repeated decoration result in only one conversion, so the user does not need to remove the manual fix even if the library ended up applying the decorator in the future.

Decorating an Airflow 3-style operator is a no-op.

Make an Airflow 3 operator compatible to 2

For an operator compatible to both Airflow 2 and 3 syntaxes, it should inherit BaseOperator from the compatibility layer instead of airflow—this keeps the operator compatible with Airflow 2, using the Airflow 3 syntax. The BaseOperator in the compatibility layer adds additional logic during resolution to correctly handle the new template classes. This way, we can avoid carrying compatibility code in Airflow 3, but still allow the user to slowly migrate to the new syntax after upgrading first (to receive other benefits).

I have not looked into if we could use a decorator for this instead of changing the base class. It might be possible, and likely preferrable if so. We’ll see during implementation.

Other migrating options

Instead of migrating operators one by one, a possibility is to have a helper function for the user to call in their cluster policies, either in dag_policy or task_policy, that dynamically replaces values in templated fields on the operator with a Template subclass. This may be the least intrusive solution—you virtually don’t need to rewrite any DAGs at all, but only add a function call in the policies—but personally I’m not sure I like the “magic” aspect of this, or want to recommend it as an official migration plan.

The literal helper will stay in core Airflow 3. It will simply does nothing when used in operators implemented in the Airflow 3 style, although the compatibility layer should still use it the same way as in Airflow 2.

Scope

This AIP should be considered done when the proposed new types are implemented, a compatibility layer is introduced, and operators in core Airflow and official providers are migrated to use the compatibility layer so they work in both Airflow 2 (in both the old and new operator templating syntaxes) and Airflow 3 (in the new operator templating syntax).

Rejected Ideas

Using subclasses to specify the rendered result type

The possibilities of using subclasses to do this has been considered, e.g. having an IsoFormatTemplate instead of passing in a separate converter. This was abandoned because we already use subclasses to distinguish between loading from a file (FileTemplate) or a literal string (StringTemplate). Using subclasses means we’ll need two subclasses for every conversion, one for loading from file and one for string, which is more confusing and harder to maintain. As an alternative, we can provide aliases functions for users to import, such as:

from airflow.templates.converters import isodate

StringTemplate("{{ ds }}", converter=isodate)

Where isodate is simply an alias to datetime.date.fromisoformat. Custom converters can also be put in the same module.