This is an automated email from the ASF dual-hosted git repository. ephraimanierobi pushed a commit to branch v2-2-test in repository https://gitbox.apache.org/repos/asf/airflow.git
commit 68acf7ddb44f7a4c1f9588d18a83e0611230c9a9 Author: hubert-pietron <94397721+hubert-piet...@users.noreply.github.com> AuthorDate: Sun Feb 27 14:07:14 2022 +0100 Fix logging JDBC SQL error when task fails (#21540) (cherry picked from commit bc1b422e1ce3a5b170618a7a6589f8ae2fc33ad6) --- airflow/utils/log/secrets_masker.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/airflow/utils/log/secrets_masker.py b/airflow/utils/log/secrets_masker.py index c263cbb..6cdfd0e 100644 --- a/airflow/utils/log/secrets_masker.py +++ b/airflow/utils/log/secrets_masker.py @@ -145,7 +145,12 @@ class SecretsMasker(logging.Filter): return frozenset(record.__dict__).difference({'msg', 'args'}) def _redact_exception_with_context(self, exception): - exception.args = (self.redact(v) for v in exception.args) + # Exception class may not be modifiable (e.g. declared by an + # extension module such as JDBC). + try: + exception.args = (self.redact(v) for v in exception.args) + except AttributeError: + pass if exception.__context__: self._redact_exception_with_context(exception.__context__) if exception.__cause__ and exception.__cause__ is not exception.__context__: