Skip to content

Commit

Permalink
[AIRFLOW-2508] Handle non string types in Operators templatized fields (
Browse files Browse the repository at this point in the history
  • Loading branch information
galak75 authored and ashb committed Mar 5, 2019
1 parent 3b28dd5 commit 057258e
Show file tree
Hide file tree
Showing 2 changed files with 119 additions and 9 deletions.
11 changes: 2 additions & 9 deletions airflow/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@
import jinja2
import json
import logging
import numbers
import os
import pendulum
import pickle
Expand Down Expand Up @@ -2879,25 +2878,19 @@ def render_template_from_field(self, attr, content, context, jinja_env):
Renders a template from a field. If the field is a string, it will
simply render the string and return the result. If it is a collection or
nested set of collections, it will traverse the structure and render
all strings in it.
all elements in it. If the field has another type, it will return it as it is.
"""
rt = self.render_template
if isinstance(content, six.string_types):
result = jinja_env.from_string(content).render(**context)
elif isinstance(content, (list, tuple)):
result = [rt(attr, e, context) for e in content]
elif isinstance(content, numbers.Number):
result = content
elif isinstance(content, dict):
result = {
k: rt("{}[{}]".format(attr, k), v, context)
for k, v in list(content.items())}
else:
param_type = type(content)
msg = (
"Type '{param_type}' used for parameter '{attr}' is "
"not supported for templating").format(**locals())
raise AirflowException(msg)
result = content
return result

def render_template(self, attr, content, context):
Expand Down
117 changes: 117 additions & 0 deletions tests/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import time
import unittest
import urllib
import uuid
from tempfile import NamedTemporaryFile, mkdtemp

import pendulum
Expand Down Expand Up @@ -395,6 +396,122 @@ def test_render_template_field(self):
result = task.render_template('', '{{ foo }}', dict(foo='bar'))
self.assertEqual(result, 'bar')

def test_render_template_list_field(self):
"""Tests if render_template from a list field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

self.assertListEqual(
task.render_template('', ['{{ foo }}_1', '{{ foo }}_2'], {'foo': 'bar'}),
['bar_1', 'bar_2']
)

def test_render_template_tuple_field(self):
"""Tests if render_template from a tuple field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

# tuple is replaced by a list
self.assertListEqual(
task.render_template('', ('{{ foo }}_1', '{{ foo }}_2'), {'foo': 'bar'}),
['bar_1', 'bar_2']
)

def test_render_template_dict_field(self):
"""Tests if render_template from a dict field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

self.assertDictEqual(
task.render_template('', {'key1': '{{ foo }}_1', 'key2': '{{ foo }}_2'}, {'foo': 'bar'}),
{'key1': 'bar_1', 'key2': 'bar_2'}
)

def test_render_template_dict_field_with_templated_keys(self):
"""Tests if render_template from a dict field works as expected:
dictionary keys are not templated"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

self.assertDictEqual(
task.render_template('', {'key_{{ foo }}_1': 1, 'key_2': '{{ foo }}_2'}, {'foo': 'bar'}),
{'key_{{ foo }}_1': 1, 'key_2': 'bar_2'}
)

def test_render_template_date_field(self):
"""Tests if render_template from a date field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

self.assertEqual(
task.render_template('', datetime.date(2018, 12, 6), {'foo': 'bar'}),
datetime.date(2018, 12, 6)
)

def test_render_template_datetime_field(self):
"""Tests if render_template from a datetime field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

self.assertEqual(
task.render_template('', datetime.datetime(2018, 12, 6, 10, 55), {'foo': 'bar'}),
datetime.datetime(2018, 12, 6, 10, 55)
)

def test_render_template_UUID_field(self):
"""Tests if render_template from a UUID field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

random_uuid = uuid.uuid4()
self.assertIs(
task.render_template('', random_uuid, {'foo': 'bar'}),
random_uuid
)

def test_render_template_object_field(self):
"""Tests if render_template from an object field works"""

dag = DAG('test-dag',
start_date=DEFAULT_DATE)

with dag:
task = DummyOperator(task_id='op1')

test_object = object()
self.assertIs(
task.render_template('', test_object, {'foo': 'bar'}),
test_object
)

def test_render_template_field_macro(self):
""" Tests if render_template from a field works,
if a custom filter was defined"""
Expand Down

0 comments on commit 057258e

Please sign in to comment.