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

Mask sensitive values for not-yet-running TIs #23807

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 11 additions & 13 deletions airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -2085,7 +2085,10 @@ def get_prev_ds_nodash() -> Optional[str]:

@provide_session
def get_rendered_template_fields(self, session: Session = NEW_SESSION) -> None:
"""Fetch rendered template fields from DB"""
"""
Update task with rendered template fields for presentation in UI.
If task has already run, will fetch from DB; otherwise will render.
"""
from airflow.models.renderedtifields import RenderedTaskInstanceFields

rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self, session=session)
Expand All @@ -2096,25 +2099,20 @@ def get_rendered_template_fields(self, session: Session = NEW_SESSION) -> None:
return

try:
# Task was never executed. Initialize RenderedTaskInstanceFields
# to render template and mask secrets. Set MASK_SECRETS_IN_LOGS
# to True to enable masking similar to task run.
original_value = settings.MASK_SECRETS_IN_LOGS
settings.MASK_SECRETS_IN_LOGS = True
rendered_task_instance = RenderedTaskInstanceFields(self)
rendered_fields = rendered_task_instance.rendered_fields
if rendered_fields:
for field_name, rendered_value in rendered_fields.items():
setattr(self.task, field_name, rendered_value)
# If we get here, either the task hasn't run or the RTIF record was purged.
from airflow.utils.log.secrets_masker import redact

self.render_templates()
for field_name in self.task.template_fields:
rendered_value = getattr(self.task, field_name)
setattr(self.task, field_name, redact(rendered_value, field_name))
except (TemplateAssertionError, UndefinedError) as e:
raise AirflowException(
"Webserver does not have access to User-defined Macros or Filters "
"when Dag Serialization is enabled. Hence for the task that have not yet "
"started running, please use 'airflow tasks render' for debugging the "
"rendering of template_fields."
) from e
finally:
settings.MASK_SECRETS_IN_LOGS = original_value

@provide_session
def get_rendered_k8s_spec(self, session=NEW_SESSION):
Expand Down
9 changes: 5 additions & 4 deletions airflow/utils/log/secrets_masker.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import re
from typing import TYPE_CHECKING, Any, Dict, Iterable, List, Optional, Set, Tuple, Union

from airflow import settings
from airflow.compat.functools import cache, cached_property

if TYPE_CHECKING:
Expand Down Expand Up @@ -84,12 +85,9 @@ def mask_secret(secret: Union[str, dict, Iterable], name: Optional[str] = None)
If ``secret`` is a dict or a iterable (excluding str) then it will be
recursively walked and keys with sensitive names will be hidden.
"""
# Delay import
from airflow import settings

# Filtering all log messages is not a free process, so we only do it when
# running tasks
if not settings.MASK_SECRETS_IN_LOGS or not secret:
if not secret:
return

_secrets_masker().add_mask(secret, name)
Expand Down Expand Up @@ -161,6 +159,9 @@ def _redact_exception_with_context(self, exception):
self._redact_exception_with_context(exception.__cause__)

def filter(self, record) -> bool:
if settings.MASK_SECRETS_IN_LOGS is not True:
return True

if self.ALREADY_FILTERED_FLAG in record.__dict__:
# Filters are attached to multiple handlers and logs, keep a
# "private" flag that stops us needing to process it more than once
Expand Down
3 changes: 3 additions & 0 deletions tests/utils/log/test_secrets_masker.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,12 @@

import pytest

from airflow import settings
from airflow.utils.log.secrets_masker import SecretsMasker, should_hide_value_for_key
from tests.test_utils.config import conf_vars

settings.MASK_SECRETS_IN_LOGS = True

p = "password"


Expand Down