Skip to content

Commit 67cbb0f

Browse files
authored
Enable specifying dictionary paths in template_fields_renderers (#17321)
Added the handling of paths in `template_fields_renderers` which enables information contained in dictionaries to be unpacked and rendered appropriately.
1 parent 97428ef commit 67cbb0f

File tree

5 files changed

+107
-5
lines changed

5 files changed

+107
-5
lines changed

β€Žairflow/providers/google/cloud/operators/bigquery.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2196,7 +2196,7 @@ class BigQueryInsertJobOperator(BaseOperator):
21962196
"impersonation_chain",
21972197
)
21982198
template_ext = (".json",)
2199-
template_fields_renderers = {"configuration": "json"}
2199+
template_fields_renderers = {"configuration": "json", "configuration.query.query": "sql"}
22002200
ui_color = BigQueryUIColors.QUERY.value
22012201

22022202
def __init__(

β€Žairflow/www/views.py

Lines changed: 45 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -279,6 +279,29 @@ def task_group_to_dict(task_group):
279279
}
280280

281281

282+
def get_key_paths(input_dict):
283+
"""Return a list of dot-separated dictionary paths"""
284+
for key, value in input_dict.items():
285+
if isinstance(value, dict):
286+
for sub_key in get_key_paths(value):
287+
yield '.'.join((key, sub_key))
288+
else:
289+
yield key
290+
291+
292+
def get_value_from_path(key_path, content):
293+
"""Return the value from a dictionary based on dot-separated path of keys"""
294+
elem = content
295+
for x in key_path.strip(".").split("."):
296+
try:
297+
x = int(x)
298+
elem = elem[x]
299+
except ValueError:
300+
elem = elem.get(x)
301+
302+
return elem
303+
304+
282305
def dag_edges(dag):
283306
"""
284307
Create the list of edges needed to construct the Graph view.
@@ -995,11 +1018,31 @@ def rendered_templates(self):
9951018
renderer = task.template_fields_renderers.get(template_field, template_field)
9961019
if renderer in renderers:
9971020
if isinstance(content, (dict, list)):
998-
content = json.dumps(content, sort_keys=True, indent=4)
999-
html_dict[template_field] = renderers[renderer](content)
1021+
json_content = json.dumps(content, sort_keys=True, indent=4)
1022+
html_dict[template_field] = renderers[renderer](json_content)
1023+
else:
1024+
html_dict[template_field] = renderers[renderer](content)
10001025
else:
10011026
html_dict[template_field] = Markup("<pre><code>{}</pre></code>").format(pformat(content))
10021027

1028+
if isinstance(content, dict):
1029+
if template_field == 'op_kwargs':
1030+
for key, value in content.items():
1031+
renderer = task.template_fields_renderers.get(key, key)
1032+
if renderer in renderers:
1033+
html_dict['.'.join([template_field, key])] = renderers[renderer](value)
1034+
else:
1035+
html_dict['.'.join([template_field, key])] = Markup(
1036+
"<pre><code>{}</pre></code>"
1037+
).format(pformat(value))
1038+
else:
1039+
for dict_keys in get_key_paths(content):
1040+
template_path = '.'.join((template_field, dict_keys))
1041+
renderer = task.template_fields_renderers.get(template_path, template_path)
1042+
if renderer in renderers:
1043+
content_value = get_value_from_path(dict_keys, content)
1044+
html_dict[template_path] = renderers[renderer](content_value)
1045+
10031046
return self.render_template(
10041047
'airflow/ti_code.html',
10051048
html_dict=html_dict,

β€Ždocs/apache-airflow/howto/custom-operator.rst

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,7 @@ with actual value. Note that Jinja substitutes the operator attributes and not t
195195
196196
In the example, the ``template_fields`` should be ``['guest_name']`` and not ``['name']``
197197

198-
Additionally you may provide ``template_fields_renderers`` dictionary which defines in what style the value
198+
Additionally you may provide ``template_fields_renderers`` a dictionary which defines in what style the value
199199
from template field renders in Web UI. For example:
200200

201201
.. code-block:: python
@@ -208,12 +208,48 @@ from template field renders in Web UI. For example:
208208
super().__init__(**kwargs)
209209
self.request_body = request_body
210210
211+
In the situation where ``template_field`` is itself a dictionary, it is also possible to specify a
212+
dot-separated key path to extract and render individual elements appropriately. For example:
213+
214+
.. code-block:: python
215+
216+
class MyConfigOperator(BaseOperator):
217+
template_fields = ["configuration"]
218+
template_fields_renderers = {
219+
"configuration": "json",
220+
"configuration.query.sql": "sql",
221+
}
222+
223+
def __init__(self, configuration: dict, **kwargs) -> None:
224+
super().__init__(**kwargs)
225+
self.configuration = configuration
226+
227+
Then using this template as follows:
228+
229+
.. code-block:: python
230+
231+
with dag:
232+
config_task = MyConfigOperator(
233+
task_id="task_id_1",
234+
configuration={"query": {"job_id": "123", "sql": "select * from my_table"}},
235+
dag=dag,
236+
)
237+
238+
This will result in the UI rendering ``configuration`` as json in addition to the value contained in the
239+
configuration at ``query.sql`` to be rendered with the SQL lexer.
240+
241+
.. image:: ../img/template_field_renderer_path.png
242+
211243
Currently available lexers:
212244

213245
- bash
214246
- doc
247+
- hql
248+
- html
249+
- jinja
215250
- json
216251
- md
252+
- powershell
217253
- py
218254
- rst
219255
- sql
62.9 KB
Loading

β€Žtests/www/views/test_views.py

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@
2222

2323
from airflow.configuration import initialize_config
2424
from airflow.plugins_manager import AirflowPlugin, EntryPointSource
25-
from airflow.www.views import get_safe_url, truncate_task_duration
25+
from airflow.www.views import get_key_paths, get_safe_url, get_value_from_path, truncate_task_duration
2626
from tests.test_utils.config import conf_vars
2727
from tests.test_utils.mock_plugins import mock_plugin_manager
2828
from tests.test_utils.www import check_content_in_response, check_content_not_in_response
@@ -243,3 +243,26 @@ def get_task_instance(session, task):
243243
dagrun.refresh_from_db(session=session)
244244
# dagrun should be set to QUEUED
245245
assert dagrun.get_state() == State.QUEUED
246+
247+
248+
TEST_CONTENT_DICT = {"key1": {"key2": "val2", "key3": "val3", "key4": {"key5": "val5"}}}
249+
250+
251+
@pytest.mark.parametrize(
252+
"test_content_dict, expected_paths", [(TEST_CONTENT_DICT, ("key1.key2", "key1.key3", "key1.key4.key5"))]
253+
)
254+
def test_generate_key_paths(test_content_dict, expected_paths):
255+
for key_path in get_key_paths(test_content_dict):
256+
assert key_path in expected_paths
257+
258+
259+
@pytest.mark.parametrize(
260+
"test_content_dict, test_key_path, expected_value",
261+
[
262+
(TEST_CONTENT_DICT, "key1.key2", "val2"),
263+
(TEST_CONTENT_DICT, "key1.key3", "val3"),
264+
(TEST_CONTENT_DICT, "key1.key4.key5", "val5"),
265+
],
266+
)
267+
def test_get_value_from_path(test_content_dict, test_key_path, expected_value):
268+
assert expected_value == get_value_from_path(test_key_path, test_content_dict)

0 commit comments

Comments
 (0)