You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by as...@apache.org on 2021/06/22 13:46:02 UTC
[airflow] 12/38: Don't fail to log if we can't redact something
(#16118)
This is an automated email from the ASF dual-hosted git repository.
ash pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git
commit 7603ef612c1f7cfdf9b85704895cad8d67c149f1
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Mon Jun 7 09:27:01 2021 +0100
Don't fail to log if we can't redact something (#16118)
Rather than dying with an exception, catch it and warn about that,
asking users to report it to us.
Additionally handle the specific case where a file handle/IO object is
logged -- we definitely don't want to iterate over that!
(cherry picked from commit 57bd6fb2925a7d505a80b83140811b94b363f49c)
---
airflow/utils/log/secrets_masker.py | 53 ++++++++++++++++++++++------------
tests/utils/log/test_secrets_masker.py | 24 +++++++++++++++
2 files changed, 59 insertions(+), 18 deletions(-)
diff --git a/airflow/utils/log/secrets_masker.py b/airflow/utils/log/secrets_masker.py
index 6df8d39..b3ccfdb 100644
--- a/airflow/utils/log/secrets_masker.py
+++ b/airflow/utils/log/secrets_masker.py
@@ -16,6 +16,7 @@
# under the License.
"""Mask sensitive information from logs"""
import collections
+import io
import logging
import re
from typing import TYPE_CHECKING, Iterable, Optional, Set, TypeVar, Union
@@ -40,6 +41,10 @@ if TYPE_CHECKING:
RedactableItem = TypeVar('RedctableItem')
+
+log = logging.getLogger(__name__)
+
+
DEFAULT_SENSITIVE_FIELDS = frozenset(
{
'password',
@@ -186,24 +191,36 @@ class SecretsMasker(logging.Filter):
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 structure.
- 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):
- return list(self.redact(subval) for subval in item)
- else:
+ try:
+ 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 structure.
+ 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, io.IOBase):
+ return item
+ elif isinstance(item, Iterable):
+ return list(self.redact(subval) for subval in item)
+ else:
+ return item
+ except Exception as e: # pylint: disable=broad-except
+ log.warning(
+ "Unable to redact %r, please report this via <https://github.com/apache/airflow/issues>. "
+ "Error was: %s: %s",
+ item,
+ type(e).__name__,
+ str(e),
+ )
return item
# pylint: enable=too-many-return-statements
diff --git a/tests/utils/log/test_secrets_masker.py b/tests/utils/log/test_secrets_masker.py
index 8c88bdd..24e86c1 100644
--- a/tests/utils/log/test_secrets_masker.py
+++ b/tests/utils/log/test_secrets_masker.py
@@ -72,6 +72,22 @@ class TestSecretsMasker:
assert caplog.text == "INFO Cannot connect to user:***\n"
+ def test_non_redactable(self, logger, caplog):
+ class NonReactable:
+ def __iter__(self):
+ raise RuntimeError("force fail")
+
+ def __repr__(self):
+ return "<NonRedactable>"
+
+ logger.info("Logging %s", NonReactable())
+
+ assert caplog.messages == [
+ "Unable to redact <NonRedactable>, please report this via "
+ + "<https://github.com/apache/airflow/issues>. Error was: RuntimeError: force fail",
+ "Logging <NonRedactable>",
+ ]
+
def test_extra(self, logger, caplog):
logger.handlers[0].formatter = ShortExcFormatter("%(levelname)s %(message)s %(conn)s")
logger.info("Cannot connect", extra={'conn': "user:password"})
@@ -202,6 +218,14 @@ class TestSecretsMasker:
assert filt.redact(value, name) == expected
+ def test_redact_filehandles(self, caplog):
+ filt = SecretsMasker()
+ with open("/dev/null", "w") as handle:
+ assert filt.redact(handle, None) == handle
+
+ # We shouldn't have logged a warning here
+ assert caplog.messages == []
+
class TestShouldHideValueForKey:
@pytest.mark.parametrize(