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

logging error #19816

Closed
1 of 2 tasks
aresabalo opened this issue Nov 24, 2021 · 9 comments · Fixed by #20039
Closed
1 of 2 tasks

logging error #19816

aresabalo opened this issue Nov 24, 2021 · 9 comments · Fixed by #20039
Labels
area:core kind:bug This is a clearly a bug

Comments

@aresabalo
Copy link

aresabalo commented Nov 24, 2021

Apache Airflow version

2.2.2 (latest released)

Operating System

Linux-5.4.0-1056-azure-x86_64-with-glibc2.2.5

Versions of Apache Airflow Providers

No response

Deployment

Other Docker-based deployment

Deployment details

kubernetes 1.21.2 (AKS)

What happened

DAG using logging class produce stack overflow and abort task. It seems like a bug in /home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py

ENV:
AIRFLOW__CORE__HIDE_SENSITIVE_VAR_CONN_FIELDS="False"

Dag Code fragment:

log_stream = StringIO()
logging.basicConfig(stream=log_stream, level=logging.DEBUG)

logger = logging.getLogger("airflow.task")
logger.addHandler(logging.StreamHandler(log_stream))
...
logger.info("------------------------------ ------------------- ------------------------------")

Log in worker pod:

Fatal Python error: Cannot recover from stack overflow.
Python runtime state: initialized

Current thread 0x00007fbf8d8190c0 (most recent call first):
  File "/usr/local/lib/python3.8/posixpath.py", line 42 in _get_sep
  File "/usr/local/lib/python3.8/posixpath.py", line 143 in basename
  File "/usr/local/lib/python3.8/logging/__init__.py", line 322 in __init__
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1556 in makeRecord
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1587 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 950 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1661 in callHandlers
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1599 in handle
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1589 in _log
  File "/usr/local/lib/python3.8/logging/__init__.py", line 1458 in warning
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 215 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in <genexpr>
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 208 in _redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 231 in redact
  File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py", line 164 in filter
  File "/usr/local/lib/python3.8/logging/__init__.py", line 811 in filter

What you expected to happen

No response

How to reproduce

No response

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@aresabalo aresabalo added area:core kind:bug This is a clearly a bug labels Nov 24, 2021
@boring-cyborg
Copy link

boring-cyborg bot commented Nov 24, 2021

Thanks for opening your first issue here! Be sure to follow the issue template!

@aresabalo
Copy link
Author

Bad workaround patch /home/airflow/.local/lib/python3.8/site-packages/airflow/utils/log/secrets_masker.py

def redact(self, item: "RedactableItem", name: Optional[str] = None) -> "RedactableItem":
    """Redact an any secrets found in ``item``, if it is a string.

    If ``name`` is given, and it's a "sensitive" name (see
    :func:`should_hide_value_for_key`) then all string values in the item
    is redacted.
    """
    # Bad workaround avoiding stack overflow
    return item
    # return self._redact(item, name, depth=0)

@uranusjr
Copy link
Member

The redact function needs to have a hard recursion limit.

@aresabalo
Copy link
Author

The redact function already has a hard recursion limit (by default 5) in secrets_masker.py
There must be another problem in that function.

Setting AIRFLOW__CORE__HIDE_SENSITIVE_VAR_CONN_FIELDS to "False" should avoid executing this buggy function, but it doesn't work either :-(


def _redact(self, item: "RedactableItem", name: Optional[str], depth: int) -> "RedactableItem":
    # Avoid spending too much effort on redacting on deeply nested
    # structures. This also avoid infinite recursion if a structure has
    # reference to self.
    if depth > self.MAX_RECURSION_DEPTH:
        return item

@hterik
Copy link
Contributor

hterik commented Dec 1, 2021

I've seen this happen as well, it happens when the _redact function throws an exception, which triggers a new log.warning("Unable to redact %r ... call that resets the recursion-limit counter.

Since the failing item itself is embedded into the warning-log, it will trigger the same issue again and it repeats forever.

@potiuk
Copy link
Member

potiuk commented Dec 1, 2021

AAAAARGH ! Really nice find @hterik - would you like to attempt to fix it ? I think one of the solutions would be to pass current depth to the log() call with some custom name and have the log pass it as starting depth to redact method?

But possibly we could find a nicer solution :)

potiuk pushed a commit to potiuk/airflow that referenced this issue Dec 5, 2021
When redact warning log on "unredactable" item is printed, the
log entered an infinite recursion, because the item was attempted
to be redacted again in the log.

This PR converts the item to str() - in the worst case the str
converstion will fail and raise exception - but this will be about
right - but it will not attempt to redact the item again.

Fixws: apache#19816
potiuk pushed a commit to potiuk/airflow that referenced this issue Dec 6, 2021
When redact warning log on "unredactable" item is printed, the
log entered an infinite recursion, because the item was attempted
to be redacted again in the log.

This PR converts the item to str() - in the worst case the str
converstion will fail and raise exception - but this will be about
right - but it will not attempt to redact the item again.

Fixes: apache#19816
potiuk added a commit that referenced this issue Dec 7, 2021
* Fix infinite recursion on redact log

When redact warning log on "unredactable" item is printed, the
log entered an infinite recursion, because the item was attempted
to be redacted again in the log.

This PR converts the item to str() - in the worst case the str
converstion will fail and raise exception - but this will be about
right - but it will not attempt to redact the item again.

Fixes: #19816

* Update airflow/utils/log/secrets_masker.py
jedcunningham pushed a commit that referenced this issue Dec 7, 2021
* Fix infinite recursion on redact log

When redact warning log on "unredactable" item is printed, the
log entered an infinite recursion, because the item was attempted
to be redacted again in the log.

This PR converts the item to str() - in the worst case the str
converstion will fail and raise exception - but this will be about
right - but it will not attempt to redact the item again.

Fixes: #19816

* Update airflow/utils/log/secrets_masker.py

(cherry picked from commit f441833)
@mitsos1os
Copy link

mitsos1os commented Feb 14, 2022

Just wanted to add some notes about getting the same error on the latest MWAA setup on Airflow 2.2.2. Tasks were failing instantly once they were executed with the same stack overflow error!

The problem finally was that LOG LEVEL! We were deploying it through Terraform and the mwaa module allows you to use log-level configuration up to DEBUG.

However, MWAA UI seems not to like them and doesn't display the DEBUG selection. As soon as we changed it to INFO+ the tasks started running normally.

I hope this saves someone all the frustration that I went through while searching for the cause of a fresh cluster not working

UPDATE
It is even stated in their CLI update utility and does not work: https://awscli.amazonaws.com/v2/documentation/api/latest/reference/mwaa/update-environment.html

@djpate
Copy link

djpate commented Mar 14, 2022

@mitsos1os You just saved my life. Thanks for sharing. Been stuck on this for two days just because of DEBUG level.

@mitsos1os
Copy link

@mitsos1os You just saved my life. Thanks for sharing. Been stuck on this for two days just because of DEBUG level.

Glad I was helpful!

Usiel added a commit to Usiel/airflow that referenced this issue Nov 22, 2023
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: apache#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)

```
potiuk pushed a commit that referenced this issue Nov 22, 2023
* Fix for infinite recursion due to secrets_masker

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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
ephraimbuddy pushed a commit that referenced this issue Nov 23, 2023
* Fix for infinite recursion due to secrets_masker

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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
ephraimbuddy pushed a commit that referenced this issue Nov 26, 2023
* Fix for infinite recursion due to secrets_masker

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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jul 18, 2024
* Fix for infinite recursion due to secrets_masker

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: apache/airflow#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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
GitOrigin-RevId: 6e8f646bf9aa071154069bfcace22e53b4d35574
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 20, 2024
* Fix for infinite recursion due to secrets_masker

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: apache/airflow#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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
GitOrigin-RevId: 6e8f646bf9aa071154069bfcace22e53b4d35574
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 8, 2024
* Fix for infinite recursion due to secrets_masker

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: apache/airflow#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)

```

* Improve error message

---------

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>
GitOrigin-RevId: 6e8f646bf9aa071154069bfcace22e53b4d35574
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:core kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

6 participants