This is an automated email from the ASF dual-hosted git repository.

dstandish pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new 2dc806367c Mask sensitive values for not-yet-running TIs (#23807)
2dc806367c is described below

commit 2dc806367c3dc27df5db4b955d151e789fbc78b0
Author: Daniel Standish <15932138+dstand...@users.noreply.github.com>
AuthorDate: Sat May 21 08:36:12 2022 -0700

    Mask sensitive values for not-yet-running TIs (#23807)
    
    Alternative approach to #22754.  Resolves  #22738.
---
 airflow/models/taskinstance.py         | 24 +++++++++++-------------
 airflow/utils/log/secrets_masker.py    |  9 +++++----
 tests/utils/log/test_secrets_masker.py |  3 +++
 3 files changed, 19 insertions(+), 17 deletions(-)

diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py
index 4b2b2f9348..e160153af7 100644
--- a/airflow/models/taskinstance.py
+++ b/airflow/models/taskinstance.py
@@ -2085,7 +2085,10 @@ class TaskInstance(Base, LoggingMixin):
 
     @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)
@@ -2096,16 +2099,13 @@ class TaskInstance(Base, LoggingMixin):
             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 "
@@ -2113,8 +2113,6 @@ class TaskInstance(Base, LoggingMixin):
                 "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):
diff --git a/airflow/utils/log/secrets_masker.py 
b/airflow/utils/log/secrets_masker.py
index 7823bf7319..c4c1c390b2 100644
--- a/airflow/utils/log/secrets_masker.py
+++ b/airflow/utils/log/secrets_masker.py
@@ -20,6 +20,7 @@ import logging
 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:
@@ -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)
@@ -161,6 +159,9 @@ class SecretsMasker(logging.Filter):
             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
diff --git a/tests/utils/log/test_secrets_masker.py 
b/tests/utils/log/test_secrets_masker.py
index 268e1dfcae..4f91da61d8 100644
--- a/tests/utils/log/test_secrets_masker.py
+++ b/tests/utils/log/test_secrets_masker.py
@@ -22,9 +22,12 @@ import textwrap
 
 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"
 
 

Reply via email to