Skip to content

Commit

Permalink
Fix for infinite recursion due to secrets_masker
Browse files Browse the repository at this point in the history
We can get into trouble for types that cannot be initiated with re2's `type(obj)()` call. The `secrets_masker` thus fails, which triggers a warning log, which also fails because we pass the object to the logger, which is then masked again, and so forth.

We can break the recursion by emitting a log without trying to redact the value again (this ensures no new bug will cause a stack overflow). This issue has occured previously: #19816 (comment)
Additionally, we fix this particular bug by ensuring whatever re2 receives is a simple `str`.

I noticed this issue while working with a DAG that calls Airflow's DB cleanup function.

Example DAG:
```
from datetime import datetime

from airflow import DAG
from airflow.models import Variable
from airflow.operators.python import PythonOperator

class MyStringClass(str):
    def __init__(self, required_arg):
        pass

def fail(task_instance):
    # make sure the `SecretsMasker` has a replacer
    Variable.set(key="secret", value="secret_value")
    Variable.get("secret")
    # trigger the infinite recursion
    task_instance.log.info("%s", MyStringClass("secret_value"))

with DAG(
    dag_id="secrets_masker_recursion",
    start_date=datetime(2023, 9, 26),
):
    PythonOperator(task_id="fail", python_callable=fail)

```
  • Loading branch information
Usiel committed Nov 22, 2023
1 parent f4e5571 commit fdd568a
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 4 deletions.
9 changes: 5 additions & 4 deletions airflow/utils/log/secrets_masker.py
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ def _redact(self, item: Redactable, name: str | None, depth: int, max_depth: int
# 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 self.replacer.sub("***", str(item))
return item
elif isinstance(item, (tuple, set)):
# Turn set in to tuple!
Expand All @@ -276,14 +276,15 @@ def _redact(self, item: Redactable, name: str | None, depth: int, max_depth: int
return item
# I think this should never happen, but it does not hurt to leave it just in case
# Well. It happened (see https://github.com/apache/airflow/issues/19816#issuecomment-983311373)
# but it caused infinite recursion, so we need to cast it to str first.
# but it caused infinite recursion, to avoid this we mark the log as already filtered.
except Exception as exc:
log.warning(
"Unable to redact %r, please report this via <https://github.com/apache/airflow/issues>. "
"Error was: %s: %s",
"Unable to redact type %s, please report this via "
"<https://github.com/apache/airflow/issues>. Error was: %s: %s",
item,
type(exc).__name__,
exc,
extra={self.ALREADY_FILTERED_FLAG: True},
)
return item

Expand Down
18 changes: 18 additions & 0 deletions tests/utils/log/test_secrets_masker.py
Original file line number Diff line number Diff line change
Expand Up @@ -305,6 +305,24 @@ def test_redact_max_depth(self, val, expected, max_depth):
got = redact(val, max_depth=max_depth)
assert got == expected

def test_redact_with_str_type(self, logger, caplog):
"""
SecretsMasker's re2 replacer has issues handling a redactable item of type
`str` with required constructor args. This test ensures there is a shim in
place that avoids any issues.
See: https://github.com/apache/airflow/issues/19816#issuecomment-983311373
"""

class StrLikeClassWithRequiredConstructorArg(str):
def __init__(self, required_arg):
pass

text = StrLikeClassWithRequiredConstructorArg("password")
logger.info("redacted: %s", text)

# we expect the object's __str__() output to be logged (no warnings due to a failed masking)
assert caplog.messages == ["redacted: ***"]

@pytest.mark.parametrize(
"state, expected",
[
Expand Down

0 comments on commit fdd568a

Please sign in to comment.