You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/05/04 13:33:57 UTC

[GitHub] [airflow] uranusjr commented on a change in pull request #15599: Mask passwords and sensitive info in task logs and UI

uranusjr commented on a change in pull request #15599:
URL: https://github.com/apache/airflow/pull/15599#discussion_r625780828



##########
File path: airflow/utils/log/secrets_masker.py
##########
@@ -0,0 +1,221 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import collections
+import logging
+import re
+from typing import TYPE_CHECKING, Iterable, Optional, Set, TypeVar, Union
+
+try:
+    # 3.8+
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+
+try:
+    # 3.9+
+    from functools import cache
+except ImportError:
+    from functools import lru_cache
+
+    cache = lru_cache(maxsize=None)
+
+
+if TYPE_CHECKING:
+    from airflow.typing_compat import RePatternType
+
+    RedactableItem = TypeVar('RedctableItem')
+
+DEFAULT_SENSITIVE_FIELDS = frozenset(
+    {
+        'password',
+        'secret',
+        'passwd',
+        'authorization',
+        'api_key',
+        'apikey',
+        'access_token',
+    }
+)
+
+
+@cache
+def get_sensitive_variables_fields():
+    """Get comma-separated sensitive Variable Fields from airflow.cfg."""
+    from airflow.configuration import conf
+
+    sensitive_fields = DEFAULT_SENSITIVE_FIELDS.copy()
+    sensitive_variable_fields = conf.get('core', 'sensitive_var_conn_names')
+    if sensitive_variable_fields:
+        sensitive_fields |= frozenset({field.strip() for field in sensitive_variable_fields.split(',')})
+    return sensitive_fields
+
+
+def should_hide_value_for_key(name):
+    """Should the value for this given name (Variable name, or key in conn.extra_dejson) be hidden"""
+    from airflow import settings
+
+    if name and settings.HIDE_SENSITIVE_VAR_CONN_FIELDS:
+        name = name.strip().lower()
+        return any(s in name for s in get_sensitive_variables_fields())
+    return False
+
+
+def mask_secret(secret: Union[str, dict, Iterable], name: str = None) -> None:
+    """
+    Mask a secret from appearing in the task logs.
+
+    If ``name`` is provided, then it will only be masked if the name matches
+    one of the configured "sensitive" names.
+
+    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:
+        return
+
+    _secrets_masker().add_mask(secret, name)
+
+
+def redact(value: "RedactableItem", name: str = None) -> "RedactableItem":
+    """Redact any secrets found in ``value``."""
+    return _secrets_masker().redact(value, name)
+
+
+@cache
+def _secrets_masker() -> "SecretsMasker":
+
+    for flt in logging.getLogger('airflow.task').filters:
+        if isinstance(flt, SecretsMasker):
+            return flt
+    raise RuntimeError("No SecretsMasker found!")
+
+
+class SecretsMasker(logging.Filter):
+    """Redact secrets from logs"""
+
+    replacer: Optional["RePatternType"] = None
+    patterns: Set[str]
+
+    ALREADY_FILTERED_FLAG = "__SecretsMasker_filtered"
+
+    def __init__(self):
+        super().__init__()
+        self.patterns = set()
+
+    @cached_property
+    def _record_attrs_to_ignore(self) -> Iterable[str]:
+        # Doing log.info(..., extra={'foo': 2}) sets extra properties on
+        # record, i.e. record.foo. And we need to filter those too. Fun
+        #
+        # Create a record, and look at what attributes are on it, and ignore
+        # all the default ones!
+
+        record = logging.getLogRecordFactory()(
+            # name, level, pathname, lineno, msg, args, exc_info, func=None, sinfo=None,
+            "x",
+            logging.INFO,
+            __file__,
+            1,
+            "",
+            tuple(),
+            exc_info=None,
+            func="funcname",
+        )
+        return frozenset(record.__dict__.keys()) - frozenset(('msg', 'args'))

Review comment:
       Man, I really hate how this looks, but it seems there really isn’t a better way. Keys in `extra` are lost in `Logger` and unless we overwrite `Logger.manager` (which would *greatly* complicate things) there isn’t a way to get the information.
   
   As a nitpick, it’s probably easier to read (and maybe slightly more performant) to do
   
   ```python
   frozenset(record.__dict__).difference(("msg", "args"))
   ```
   
   (But maybe it’s just I’ve never properly learned to read set operators.)

##########
File path: docs/apache-airflow/security/secrets/index.rst
##########
@@ -30,6 +30,66 @@ The following are particularly protected:
 .. toctree::
     :maxdepth: 1
     :glob:
+    :caption: Further reading:
 
-    fernet
-    secrets-backend/index
+    Encryption at rest <fernet>
+    Using external Secret stores <secrets-backend/index>
+
+.. _security:mask-sensitive-values:
+
+Masking sensitive data
+----------------------
+
+Airflow will by default mask Connection passwords and sensitive Variables and keys from a Connection's
+extra (JSON) field when they appear in Task logs, or in the Variable view of the UI.
+
+It does this by looking for the specific *value* appearing anywhere in your output. This means that if you
+have a connection with a password of ``a``, then every instance of the letter a in your logs will be repleaced
+with ``***``.
+
+To disable masking you can setting :ref:`config:core__hide_sensitive_var_conn_fields` to False.
+
+Sensitive field names
+"""""""""""""""""""""
+
+When masking is enabled, Airflow will always mask the password field of every Connection that is accessed by a
+task.
+
+It will also mask the value of a Variable, or the field of a Connection's extra JSON blob if the name contains
+any words in ('password', 'secret', 'passwd', 'authorization', 'api_key', 'apikey', 'access_token'). This list
+can also be extended:
+
+.. code-block:: ini
+
+    [core]
+    sensitive_var_conn_names = comma,separated,sensitive,names
+
+Adding your own masks
+"""""""""""""""""""""
+
+If you want to mask an additional secret that is already masked by one of the above methods, you can do it in
+your DAG file or operator's ``execute`` function using the ``mask_secret`` function. For example:
+
+.. code-block:: python
+
+    @task
+    def my_func():
+        from airflow.utils.log.secrets_masker import mask_secret

Review comment:
       This doesn’t feel like a top level import to me. Maybe the import path should be shorter, but probably not *that* short.

##########
File path: airflow/www/views.py
##########
@@ -3227,9 +3227,11 @@ class VariableModelView(AirflowModelView):
 
     def hidden_field_formatter(self):
         """Formats hidden fields"""
+        from airflow.utils.log import secrets_masker

Review comment:
       Why can’t this be imported globally?

##########
File path: airflow/utils/log/secrets_masker.py
##########
@@ -0,0 +1,221 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import collections
+import logging
+import re
+from typing import TYPE_CHECKING, Iterable, Optional, Set, TypeVar, Union
+
+try:
+    # 3.8+
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+
+try:
+    # 3.9+
+    from functools import cache
+except ImportError:
+    from functools import lru_cache
+
+    cache = lru_cache(maxsize=None)
+
+
+if TYPE_CHECKING:
+    from airflow.typing_compat import RePatternType
+
+    RedactableItem = TypeVar('RedctableItem')
+
+DEFAULT_SENSITIVE_FIELDS = frozenset(
+    {
+        'password',
+        'secret',
+        'passwd',
+        'authorization',
+        'api_key',
+        'apikey',
+        'access_token',
+    }
+)
+
+
+@cache
+def get_sensitive_variables_fields():
+    """Get comma-separated sensitive Variable Fields from airflow.cfg."""
+    from airflow.configuration import conf
+
+    sensitive_fields = DEFAULT_SENSITIVE_FIELDS.copy()
+    sensitive_variable_fields = conf.get('core', 'sensitive_var_conn_names')
+    if sensitive_variable_fields:
+        sensitive_fields |= frozenset({field.strip() for field in sensitive_variable_fields.split(',')})
+    return sensitive_fields
+
+
+def should_hide_value_for_key(name):
+    """Should the value for this given name (Variable name, or key in conn.extra_dejson) be hidden"""
+    from airflow import settings
+
+    if name and settings.HIDE_SENSITIVE_VAR_CONN_FIELDS:
+        name = name.strip().lower()
+        return any(s in name for s in get_sensitive_variables_fields())
+    return False
+
+
+def mask_secret(secret: Union[str, dict, Iterable], name: str = None) -> None:
+    """
+    Mask a secret from appearing in the task logs.
+
+    If ``name`` is provided, then it will only be masked if the name matches
+    one of the configured "sensitive" names.
+
+    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:
+        return
+
+    _secrets_masker().add_mask(secret, name)
+
+
+def redact(value: "RedactableItem", name: str = None) -> "RedactableItem":
+    """Redact any secrets found in ``value``."""
+    return _secrets_masker().redact(value, name)
+
+
+@cache
+def _secrets_masker() -> "SecretsMasker":
+
+    for flt in logging.getLogger('airflow.task').filters:
+        if isinstance(flt, SecretsMasker):
+            return flt
+    raise RuntimeError("No SecretsMasker found!")
+
+
+class SecretsMasker(logging.Filter):
+    """Redact secrets from logs"""
+
+    replacer: Optional["RePatternType"] = None
+    patterns: Set[str]
+
+    ALREADY_FILTERED_FLAG = "__SecretsMasker_filtered"
+
+    def __init__(self):
+        super().__init__()
+        self.patterns = set()
+
+    @cached_property
+    def _record_attrs_to_ignore(self) -> Iterable[str]:
+        # Doing log.info(..., extra={'foo': 2}) sets extra properties on
+        # record, i.e. record.foo. And we need to filter those too. Fun
+        #
+        # Create a record, and look at what attributes are on it, and ignore
+        # all the default ones!
+
+        record = logging.getLogRecordFactory()(
+            # name, level, pathname, lineno, msg, args, exc_info, func=None, sinfo=None,
+            "x",
+            logging.INFO,
+            __file__,
+            1,
+            "",
+            tuple(),
+            exc_info=None,
+            func="funcname",
+        )
+        return frozenset(record.__dict__.keys()) - frozenset(('msg', 'args'))
+
+    def filter(self, record) -> bool:
+        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
+            return True
+
+        if self.replacer:
+            for k, v in record.__dict__.items():
+                if k in self._record_attrs_to_ignore:
+                    continue
+                record.__dict__[k] = self.redact(v)
+            if record.exc_info:
+                exc = record.exc_info[1]
+                # I'm not sure if this is a good idea!
+                exc.args = (self.redact(v) for v in exc.args)
+        record.__dict__[self.ALREADY_FILTERED_FLAG] = True
+
+        return True
+
+    def _redact_all(self, item: "RedactableItem") -> "RedactableItem":
+        if isinstance(item, dict):
+            return {dict_key: self._redact_all(subval) for dict_key, subval in item.items()}
+        elif isinstance(item, str):
+            return '***'
+        elif isinstance(item, (tuple, set)):
+            # Turn set in to tuple!
+            return tuple(self._redact_all(subval) for subval in item)
+        elif isinstance(item, (Iterable, list)):
+            return list(self._redact_all(subval) for subval in item)
+        else:
+            return item
+
+    # pylint: disable=too-many-return-statements
+    def redact(self, item: "RedactableItem", name: str = None) -> "RedactableItem":
+        """
+        Redact an any secrets found in ``item``, if it is a string.
+
+        If ``name`` is given, and it's a "sensitve" name (see
+        :func:`should_hide_value_for_key`) then all string values in the item
+        is redacted.
+
+        """
+        if name and should_hide_value_for_key(name):
+            return self._redact_all(item)
+
+        if isinstance(item, dict):
+            return {dict_key: self.redact(subval, dict_key) for dict_key, subval in item.items()}
+        elif isinstance(item, str):
+            if self.replacer:
+                # We can't replace specific values, but the key-based redacting
+                # can still happen, so we can't short-circuit, we need to walk
+                # the strucutre.
+                return self.replacer.sub('***', item)
+            return item
+        elif isinstance(item, (tuple, set)):
+            # Turn set in to tuple!
+            return tuple(self.redact(subval) for subval in item)
+        elif isinstance(item, (Iterable, list)):

Review comment:
       `list` is iterable by default so `isinstance(item, Iterable)` is sufficient.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org