From 76ec48d32a3d150e227cf684874c371b696bb1de Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 14:59:07 -0800 Subject: [PATCH 1/7] aggressively mask user code errors --- .../dagster/dagster/_core/errors.py | 4 +- .../dagster/_core/execution/plan/utils.py | 5 ++- .../dagster/dagster/_utils/error.py | 43 ++++++++++++++----- 3 files changed, 40 insertions(+), 12 deletions(-) diff --git a/python_modules/dagster/dagster/_core/errors.py b/python_modules/dagster/dagster/_core/errors.py index e2da56d5eb05d..e2f62109fe7d0 100644 --- a/python_modules/dagster/dagster/_core/errors.py +++ b/python_modules/dagster/dagster/_core/errors.py @@ -280,8 +280,10 @@ def user_code_error_boundary( """ check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) + from dagster._utils.error import log_and_redact_stacktrace_if_enabled - with raise_execution_interrupts(): + print("enter user") + with log_and_redact_stacktrace_if_enabled(), raise_execution_interrupts(): if log_manager: log_manager.begin_python_log_capture() try: diff --git a/python_modules/dagster/dagster/_core/execution/plan/utils.py b/python_modules/dagster/dagster/_core/execution/plan/utils.py index eaeb65a9e03c5..146a02cb3118c 100644 --- a/python_modules/dagster/dagster/_core/execution/plan/utils.py +++ b/python_modules/dagster/dagster/_core/execution/plan/utils.py @@ -42,12 +42,15 @@ def op_execution_error_boundary( as respecting the RetryPolicy if present. """ from dagster._core.execution.context.system import StepExecutionContext + from dagster._utils.error import log_and_redact_stacktrace_if_enabled + + print("enter op") check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) check.inst_param(step_context, "step_context", StepExecutionContext) - with raise_execution_interrupts(): + with log_and_redact_stacktrace_if_enabled(), raise_execution_interrupts(): step_context.log.begin_python_log_capture() retry_policy = step_context.op_retry_policy diff --git a/python_modules/dagster/dagster/_utils/error.py b/python_modules/dagster/dagster/_utils/error.py index 291bce2f0b3bd..e0c0967953c75 100644 --- a/python_modules/dagster/dagster/_utils/error.py +++ b/python_modules/dagster/dagster/_utils/error.py @@ -1,8 +1,11 @@ +import contextlib import logging import os +import sys import traceback import uuid from collections.abc import Sequence +from contextvars import ContextVar from types import TracebackType from typing import Any, NamedTuple, Optional, Union @@ -93,6 +96,31 @@ def _should_redact_user_code_error() -> bool: "DAGSTER_REDACTED_ERROR_LOGGER_NAME", "dagster.redacted_errors" ) +last_user_code_error_id: ContextVar[Optional[str]] = ContextVar( + "last_user_code_error_id", default=None +) + + +@contextlib.contextmanager +def log_and_redact_stacktrace_if_enabled(): + if not _should_redact_user_code_error(): + yield + else: + try: + yield + except BaseException as e: + exc_info = sys.exc_info() + error_id = last_user_code_error_id.get() or str(uuid.uuid4()) + masked_logger = logging.getLogger(_REDACTED_ERROR_LOGGER_NAME) + + last_user_code_error_id.set(error_id) + masked_logger.error( + f"Error occurred during user code execution, error ID {error_id}", + exc_info=exc_info, + ) + + raise e.with_traceback(None) from None + def serializable_error_info_from_exc_info( exc_info: ExceptionInfo, @@ -116,19 +144,14 @@ def serializable_error_info_from_exc_info( e = check.not_none(e, additional_message=additional_message) tb = check.not_none(tb, additional_message=additional_message) - from dagster._core.errors import DagsterUserCodeExecutionError, DagsterUserCodeProcessError + from dagster._core.errors import DagsterUserCodeProcessError - if isinstance(e, DagsterUserCodeExecutionError) and _should_redact_user_code_error(): - error_id = str(uuid.uuid4()) - masked_logger = logging.getLogger(_REDACTED_ERROR_LOGGER_NAME) - - masked_logger.error( - f"Error occurred during user code execution, error ID {error_id}", - exc_info=exc_info, - ) + err_id = last_user_code_error_id.get() + if err_id: + last_user_code_error_id.set(None) return SerializableErrorInfo( message=( - f"Error occurred during user code execution, error ID {error_id}. " + f"Error occurred during user code execution, error ID {err_id}. " "The error has been masked to prevent leaking sensitive information. " "Search in logs for this error ID for more details." ), From 7f47b4d2396ccf9677a6b86894160d90b0f51964 Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 15:21:11 -0800 Subject: [PATCH 2/7] tests --- .../dagster/dagster/_core/errors.py | 1 - .../dagster/_core/execution/plan/utils.py | 2 -- .../dagster/dagster/_utils/error.py | 29 +++++++++++-------- .../core_tests/test_mask_user_code_errors.py | 27 ++++++++++++++--- 4 files changed, 40 insertions(+), 19 deletions(-) diff --git a/python_modules/dagster/dagster/_core/errors.py b/python_modules/dagster/dagster/_core/errors.py index e2f62109fe7d0..d7dde0621b5d0 100644 --- a/python_modules/dagster/dagster/_core/errors.py +++ b/python_modules/dagster/dagster/_core/errors.py @@ -282,7 +282,6 @@ def user_code_error_boundary( check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) from dagster._utils.error import log_and_redact_stacktrace_if_enabled - print("enter user") with log_and_redact_stacktrace_if_enabled(), raise_execution_interrupts(): if log_manager: log_manager.begin_python_log_capture() diff --git a/python_modules/dagster/dagster/_core/execution/plan/utils.py b/python_modules/dagster/dagster/_core/execution/plan/utils.py index 146a02cb3118c..103a0186331d8 100644 --- a/python_modules/dagster/dagster/_core/execution/plan/utils.py +++ b/python_modules/dagster/dagster/_core/execution/plan/utils.py @@ -44,8 +44,6 @@ def op_execution_error_boundary( from dagster._core.execution.context.system import StepExecutionContext from dagster._utils.error import log_and_redact_stacktrace_if_enabled - print("enter op") - check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) check.inst_param(step_context, "step_context", StepExecutionContext) diff --git a/python_modules/dagster/dagster/_utils/error.py b/python_modules/dagster/dagster/_utils/error.py index e0c0967953c75..42537a735ff6d 100644 --- a/python_modules/dagster/dagster/_utils/error.py +++ b/python_modules/dagster/dagster/_utils/error.py @@ -4,7 +4,7 @@ import sys import traceback import uuid -from collections.abc import Sequence +from collections.abc import Mapping, Sequence from contextvars import ContextVar from types import TracebackType from typing import Any, NamedTuple, Optional, Union @@ -96,8 +96,8 @@ def _should_redact_user_code_error() -> bool: "DAGSTER_REDACTED_ERROR_LOGGER_NAME", "dagster.redacted_errors" ) -last_user_code_error_id: ContextVar[Optional[str]] = ContextVar( - "last_user_code_error_id", default=None +error_id_by_exception: ContextVar[Mapping[int, str]] = ContextVar( + "error_id_by_exception", default={} ) @@ -110,14 +110,20 @@ def log_and_redact_stacktrace_if_enabled(): yield except BaseException as e: exc_info = sys.exc_info() - error_id = last_user_code_error_id.get() or str(uuid.uuid4()) - masked_logger = logging.getLogger(_REDACTED_ERROR_LOGGER_NAME) - last_user_code_error_id.set(error_id) - masked_logger.error( - f"Error occurred during user code execution, error ID {error_id}", - exc_info=exc_info, - ) + # Generate a unique error ID for this error, or re-use an existing one + # if this error has already been seen + existing_error_id = error_id_by_exception.get().get(id(e)) + + if not existing_error_id: + error_id = str(uuid.uuid4()) + error_id_by_exception.set({**error_id_by_exception.get(), id(e): error_id}) + masked_logger = logging.getLogger(_REDACTED_ERROR_LOGGER_NAME) + + masked_logger.error( + f"Error occurred during user code execution, error ID {error_id}", + exc_info=exc_info, + ) raise e.with_traceback(None) from None @@ -146,9 +152,8 @@ def serializable_error_info_from_exc_info( from dagster._core.errors import DagsterUserCodeProcessError - err_id = last_user_code_error_id.get() + err_id = error_id_by_exception.get().get(id(e)) if err_id: - last_user_code_error_id.set(None) return SerializableErrorInfo( message=( f"Error occurred during user code execution, error ID {err_id}. " diff --git a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py index f51fa894a6149..f5b2eb93105c6 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py @@ -2,7 +2,7 @@ import sys import time import traceback -from typing import Any +from typing import Any, Callable import pytest from dagster import Config, RunConfig, config_mapping, job, op @@ -36,10 +36,24 @@ def enable_masking_user_code_errors() -> Any: yield -def test_masking_op_execution(enable_masking_user_code_errors) -> Any: +class hunter2: + pass + + +@pytest.mark.parametrize( + "build_exc", + [lambda: UserError(), lambda: KeyboardInterrupt("hunter2"), lambda: hunter2() - 5], # type:ignore + ids=["UserError", "KeyboardInterrupt", "TypeError"], +) +def test_masking_op_execution( + enable_masking_user_code_errors, build_exc: Callable[[], BaseException] +) -> Any: @op def throws_user_error(_): - raise UserError() + def hunter2(): + raise build_exc() + + hunter2() @job def job_def(): @@ -47,7 +61,12 @@ def job_def(): result = job_def.execute_in_process(raise_on_error=False) assert not result.success - assert not any("hunter2" in str(event) for event in result.all_events) + + # Ensure error message and contents of user code don't leak (e.g. hunter2 text or function name) + assert not any("hunter2" in str(event).lower() for event in result.all_events), [ + str(event) for event in result.all_events if "hunter2" in str(event) + ] + step_error = next(event for event in result.all_events if event.is_step_failure) assert ( step_error.step_failure_data.error From 057c5e1cf67beeb181442643e9252212d3a5ccc2 Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 15:38:42 -0800 Subject: [PATCH 3/7] more tests --- .../core_tests/test_mask_user_code_errors.py | 105 +++++++++++++++++- 1 file changed, 102 insertions(+), 3 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py index f5b2eb93105c6..6a70ecf571b46 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py @@ -7,7 +7,11 @@ import pytest from dagster import Config, RunConfig, config_mapping, job, op from dagster._core.definitions.timestamp import TimestampWithTimezone -from dagster._core.errors import DagsterUserCodeProcessError +from dagster._core.errors import ( + DagsterUserCodeExecutionError, + DagsterUserCodeProcessError, + user_code_error_boundary, +) from dagster._core.test_utils import environ, instance_for_test from dagster._utils.error import ( _serializable_error_info_from_tb, @@ -30,14 +34,109 @@ def __init__(self): ) +class hunter2: + pass + + @pytest.fixture(scope="function") def enable_masking_user_code_errors() -> Any: with environ({"DAGSTER_REDACT_USER_CODE_ERRORS": "1"}): yield -class hunter2: - pass +def test_masking_basic(enable_masking_user_code_errors): + try: + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "hunter2", + ): + + def hunter2(): + raise UserError() + + hunter2() + except Exception: + exc_info = sys.exc_info() + err_info = serializable_error_info_from_exc_info(exc_info) + + assert "hunter2" not in str(err_info) + + +def test_masking_nested_user_code_err_boundaries(enable_masking_user_code_errors): + try: + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "hunter2 as well", + ): + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "hunter2", + ): + + def hunter2(): + raise UserError() + + hunter2() + except Exception: + exc_info = sys.exc_info() + err_info = serializable_error_info_from_exc_info(exc_info) + + assert "hunter2" not in str(err_info) + + +def test_masking_nested_user_code_err_reraise(enable_masking_user_code_errors): + class MyException(Exception): + inner: Exception + + def __init__(self, inner: Exception): + self.inner = inner + + try: + try: + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "hunter2", + ): + + def hunter2(): + raise UserError() + + hunter2() + except Exception as e: + raise MyException(e) from e + + except Exception: + exc_info = sys.exc_info() + err_info = serializable_error_info_from_exc_info(exc_info) + + assert "hunter2" not in str(err_info) + + +def test_masking_nested_user_code_err_boundaries_reraise(enable_masking_user_code_errors): + try: + try: + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "hunter2", + ): + + def hunter2(): + raise UserError() + + hunter2() + except Exception as e: + with user_code_error_boundary( + error_cls=DagsterUserCodeExecutionError, + msg_fn=lambda: "teardown after we raised hunter2 error", + ): + # do teardown stuff + raise e + + except Exception: + exc_info = sys.exc_info() + err_info = serializable_error_info_from_exc_info(exc_info) + + assert "hunter2" not in str(err_info) @pytest.mark.parametrize( From 9da71cfbda3e92f949d12f870ef67d9e06628046 Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 15:41:48 -0800 Subject: [PATCH 4/7] update --- .../dagster_tests/core_tests/test_mask_user_code_errors.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py index 6a70ecf571b46..fe330eb264fa2 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py @@ -95,7 +95,7 @@ def __init__(self, inner: Exception): try: with user_code_error_boundary( error_cls=DagsterUserCodeExecutionError, - msg_fn=lambda: "hunter2", + msg_fn=lambda: "test", ): def hunter2(): From 67576ee74368f0791d50a8a783a3caae0cc27d6e Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 15:58:42 -0800 Subject: [PATCH 5/7] special case exception types --- .../dagster/dagster/_utils/error.py | 39 ++++++++---- .../core_tests/test_mask_user_code_errors.py | 61 ++++++++----------- 2 files changed, 53 insertions(+), 47 deletions(-) diff --git a/python_modules/dagster/dagster/_utils/error.py b/python_modules/dagster/dagster/_utils/error.py index 42537a735ff6d..4ca2c80db831d 100644 --- a/python_modules/dagster/dagster/_utils/error.py +++ b/python_modules/dagster/dagster/_utils/error.py @@ -12,6 +12,7 @@ from typing_extensions import TypeAlias import dagster._check as check +from dagster._core.errors import DagsterUserCodeExecutionError from dagster._serdes import whitelist_for_serdes @@ -154,17 +155,33 @@ def serializable_error_info_from_exc_info( err_id = error_id_by_exception.get().get(id(e)) if err_id: - return SerializableErrorInfo( - message=( - f"Error occurred during user code execution, error ID {err_id}. " - "The error has been masked to prevent leaking sensitive information. " - "Search in logs for this error ID for more details." - ), - stack=[], - cls_name="DagsterRedactedUserCodeError", - cause=None, - context=None, - ) + if isinstance(e, DagsterUserCodeExecutionError): + return SerializableErrorInfo( + message=( + f"Error occurred during user code execution, error ID {err_id}. " + "The error has been masked to prevent leaking sensitive information. " + "Search in logs for this error ID for more details." + ), + stack=[], + cls_name="DagsterRedactedUserCodeError", + cause=None, + context=None, + ) + else: + tb_exc = traceback.TracebackException(exc_type, e, tb) + error_info = _serializable_error_info_from_tb(tb_exc) + return SerializableErrorInfo( + message=error_info.message + + ( + "Error ID {err_id}. " + "The error has been masked to prevent leaking sensitive information. " + "Search in logs for this error ID for more details." + ), + stack=[], + cls_name=error_info.cls_name, + cause=None, + context=None, + ) if ( hoist_user_code_error diff --git a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py index fe330eb264fa2..17b7b44549de8 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py @@ -6,8 +6,10 @@ import pytest from dagster import Config, RunConfig, config_mapping, job, op +from dagster._core.definitions.events import Failure from dagster._core.definitions.timestamp import TimestampWithTimezone from dagster._core.errors import ( + DagsterExecutionInterruptedError, DagsterUserCodeExecutionError, DagsterUserCodeProcessError, user_code_error_boundary, @@ -84,34 +86,6 @@ def hunter2(): assert "hunter2" not in str(err_info) -def test_masking_nested_user_code_err_reraise(enable_masking_user_code_errors): - class MyException(Exception): - inner: Exception - - def __init__(self, inner: Exception): - self.inner = inner - - try: - try: - with user_code_error_boundary( - error_cls=DagsterUserCodeExecutionError, - msg_fn=lambda: "test", - ): - - def hunter2(): - raise UserError() - - hunter2() - except Exception as e: - raise MyException(e) from e - - except Exception: - exc_info = sys.exc_info() - err_info = serializable_error_info_from_exc_info(exc_info) - - assert "hunter2" not in str(err_info) - - def test_masking_nested_user_code_err_boundaries_reraise(enable_masking_user_code_errors): try: try: @@ -140,12 +114,20 @@ def hunter2(): @pytest.mark.parametrize( - "build_exc", - [lambda: UserError(), lambda: KeyboardInterrupt("hunter2"), lambda: hunter2() - 5], # type:ignore - ids=["UserError", "KeyboardInterrupt", "TypeError"], + "exc_name, expect_exc_name_in_error, build_exc", + [ + ("UserError", False, lambda: UserError()), + ("TypeError", False, lambda: TypeError("hunter2")), + ("KeyboardInterrupt", True, lambda: KeyboardInterrupt()), + ("DagsterExecutionInterruptedError", True, lambda: DagsterExecutionInterruptedError()), + ("Failure", True, lambda: Failure("asdf")), + ], ) def test_masking_op_execution( - enable_masking_user_code_errors, build_exc: Callable[[], BaseException] + enable_masking_user_code_errors, + exc_name: str, + expect_exc_name_in_error: bool, + build_exc: Callable[[], BaseException], ) -> Any: @op def throws_user_error(_): @@ -167,10 +149,17 @@ def job_def(): ] step_error = next(event for event in result.all_events if event.is_step_failure) - assert ( - step_error.step_failure_data.error - and step_error.step_failure_data.error.cls_name == "DagsterRedactedUserCodeError" - ) + + if expect_exc_name_in_error: + assert ( + step_error.step_failure_data.error + and step_error.step_failure_data.error.cls_name == exc_name + ) + else: + assert ( + step_error.step_failure_data.error + and step_error.step_failure_data.error.cls_name == "DagsterRedactedUserCodeError" + ) ERROR_ID_REGEX = r"Error occurred during user code execution, error ID ([a-z0-9\-]+)" From e8ff61366a7572858a1e69f730e4aef2fb9d09d5 Mon Sep 17 00:00:00 2001 From: benpankow Date: Thu, 16 Jan 2025 15:59:50 -0800 Subject: [PATCH 6/7] errid --- python_modules/dagster/dagster/_utils/error.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/_utils/error.py b/python_modules/dagster/dagster/_utils/error.py index 4ca2c80db831d..487d48c15c4af 100644 --- a/python_modules/dagster/dagster/_utils/error.py +++ b/python_modules/dagster/dagster/_utils/error.py @@ -173,7 +173,7 @@ def serializable_error_info_from_exc_info( return SerializableErrorInfo( message=error_info.message + ( - "Error ID {err_id}. " + f"Error ID {err_id}. " "The error has been masked to prevent leaking sensitive information. " "Search in logs for this error ID for more details." ), From c6aef4fd33e34a83ce3b7367680d3510c74640ee Mon Sep 17 00:00:00 2001 From: benpankow Date: Fri, 17 Jan 2025 11:41:48 -0800 Subject: [PATCH 7/7] comments --- python_modules/dagster/dagster/_core/errors.py | 4 ++-- .../dagster/dagster/_core/execution/plan/utils.py | 4 ++-- python_modules/dagster/dagster/_utils/error.py | 10 +++++++++- .../core_tests/test_mask_user_code_errors.py | 2 ++ 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/python_modules/dagster/dagster/_core/errors.py b/python_modules/dagster/dagster/_core/errors.py index d7dde0621b5d0..5ea13ffe99314 100644 --- a/python_modules/dagster/dagster/_core/errors.py +++ b/python_modules/dagster/dagster/_core/errors.py @@ -280,9 +280,9 @@ def user_code_error_boundary( """ check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) - from dagster._utils.error import log_and_redact_stacktrace_if_enabled + from dagster._utils.error import redact_user_stacktrace_if_enabled - with log_and_redact_stacktrace_if_enabled(), raise_execution_interrupts(): + with redact_user_stacktrace_if_enabled(), raise_execution_interrupts(): if log_manager: log_manager.begin_python_log_capture() try: diff --git a/python_modules/dagster/dagster/_core/execution/plan/utils.py b/python_modules/dagster/dagster/_core/execution/plan/utils.py index 103a0186331d8..9d0869a6817aa 100644 --- a/python_modules/dagster/dagster/_core/execution/plan/utils.py +++ b/python_modules/dagster/dagster/_core/execution/plan/utils.py @@ -42,13 +42,13 @@ def op_execution_error_boundary( as respecting the RetryPolicy if present. """ from dagster._core.execution.context.system import StepExecutionContext - from dagster._utils.error import log_and_redact_stacktrace_if_enabled + from dagster._utils.error import redact_user_stacktrace_if_enabled check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) check.inst_param(step_context, "step_context", StepExecutionContext) - with log_and_redact_stacktrace_if_enabled(), raise_execution_interrupts(): + with redact_user_stacktrace_if_enabled(), raise_execution_interrupts(): step_context.log.begin_python_log_capture() retry_policy = step_context.op_retry_policy diff --git a/python_modules/dagster/dagster/_utils/error.py b/python_modules/dagster/dagster/_utils/error.py index 487d48c15c4af..280fb5aaaf9ad 100644 --- a/python_modules/dagster/dagster/_utils/error.py +++ b/python_modules/dagster/dagster/_utils/error.py @@ -103,7 +103,12 @@ def _should_redact_user_code_error() -> bool: @contextlib.contextmanager -def log_and_redact_stacktrace_if_enabled(): +def redact_user_stacktrace_if_enabled(): + """Context manager which, if a user has enabled redacting user code errors, logs exceptions raised from within, + and clears the stacktrace from the exception. It also marks the exception to be redacted if it was to be persisted + or otherwise serialized to be sent to Dagster Plus. This is useful for preventing sensitive information from + being leaked in error messages. + """ if not _should_redact_user_code_error(): yield else: @@ -118,6 +123,8 @@ def log_and_redact_stacktrace_if_enabled(): if not existing_error_id: error_id = str(uuid.uuid4()) + + # Track the error ID for this exception so we can redact it later error_id_by_exception.set({**error_id_by_exception.get(), id(e): error_id}) masked_logger = logging.getLogger(_REDACTED_ERROR_LOGGER_NAME) @@ -126,6 +133,7 @@ def log_and_redact_stacktrace_if_enabled(): exc_info=exc_info, ) + # Redact the stacktrace to ensure it will not be passed to Dagster Plus raise e.with_traceback(None) from None diff --git a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py index 17b7b44549de8..a7fe9d16acf1a 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py @@ -99,6 +99,8 @@ def hunter2(): hunter2() except Exception as e: + # Mimics behavior of resource teardown, which runs in a + # user_code_error_boundary after the user code raises an error with user_code_error_boundary( error_cls=DagsterUserCodeExecutionError, msg_fn=lambda: "teardown after we raised hunter2 error",