This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v3-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit d66cd2e2973a4e54ff715ce0aff69eb09b32e827
Author: github-actions[bot] 
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Thu Nov 13 23:18:06 2025 +0100

    [v3-1-test] Mask kwargs on illegal args (#58252) (#58283)
    
    (cherry picked from commit 6fd0ee15e60470cd1747c4ec30b25f7a2acc65a5)
    
    Co-authored-by: Jens Scheffler <[email protected]>
---
 task-sdk/src/airflow/sdk/bases/operator.py     |  3 ++-
 task-sdk/tests/task_sdk/bases/test_operator.py | 17 +++++++++++++++++
 2 files changed, 19 insertions(+), 1 deletion(-)

diff --git a/task-sdk/src/airflow/sdk/bases/operator.py 
b/task-sdk/src/airflow/sdk/bases/operator.py
index a11b5894b0d..98bffd07dad 100644
--- a/task-sdk/src/airflow/sdk/bases/operator.py
+++ b/task-sdk/src/airflow/sdk/bases/operator.py
@@ -37,6 +37,7 @@ import attrs
 
 from airflow.exceptions import RemovedInAirflow4Warning
 from airflow.sdk import TriggerRule, timezone
+from airflow.sdk._shared.secrets_masker import redact
 from airflow.sdk.definitions._internal.abstractoperator import (
     DEFAULT_IGNORE_FIRST_DEPENDS_ON_PAST,
     DEFAULT_OWNER,
@@ -1050,7 +1051,7 @@ class BaseOperator(AbstractOperator, 
metaclass=BaseOperatorMeta):
         if kwargs:
             raise TypeError(
                 f"Invalid arguments were passed to {self.__class__.__name__} 
(task_id: {task_id}). "
-                f"Invalid arguments were:\n**kwargs: {kwargs}",
+                f"Invalid arguments were:\n**kwargs: {redact(kwargs)}",
             )
         validate_key(self.task_id)
 
diff --git a/task-sdk/tests/task_sdk/bases/test_operator.py 
b/task-sdk/tests/task_sdk/bases/test_operator.py
index c49b9db9650..6d0f4cbc0b4 100644
--- a/task-sdk/tests/task_sdk/bases/test_operator.py
+++ b/task-sdk/tests/task_sdk/bases/test_operator.py
@@ -241,6 +241,23 @@ class TestBaseOperator:
                 illegal_argument_1234="hello?",
             )
 
+    @mock.patch("airflow.sdk.bases.operator.redact")
+    def test_illegal_args_with_secrets(self, mock_redact):
+        """
+        Tests that operators on illegal arguments with secrets are correctly 
masked.
+        """
+        secret = "secretP4ssw0rd!"
+        mock_redact.side_effect = ["***"]
+
+        msg = r"Invalid arguments were passed to BaseOperator"
+        with pytest.raises(TypeError, match=msg) as exc_info:
+            BaseOperator(
+                task_id="test_illegal_args",
+                secret_argument=secret,
+            )
+        assert "***" in str(exc_info.value)
+        assert secret not in str(exc_info.value)
+
     def test_invalid_type_for_default_arg(self):
         error_msg = "'max_active_tis_per_dag' for task 'test' expects <class 
'int'>, got <class 'str'> with value 'not_an_int'"
         with pytest.raises(TypeError, match=error_msg):

Reply via email to