-
Notifications
You must be signed in to change notification settings - Fork 1.5k
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
More aggressively mask user code errors when masking enabled #27183
base: master
Are you sure you want to change the base?
Changes from all commits
76ec48d
7f47b4d
057c5e1
9da71cf
67576ee
e8ff613
c6aef4f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,14 +1,18 @@ | ||
import contextlib | ||
import logging | ||
import os | ||
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 | ||
|
||
from typing_extensions import TypeAlias | ||
|
||
import dagster._check as check | ||
from dagster._core.errors import DagsterUserCodeExecutionError | ||
from dagster._serdes import whitelist_for_serdes | ||
|
||
|
||
|
@@ -93,6 +97,45 @@ def _should_redact_user_code_error() -> bool: | |
"DAGSTER_REDACTED_ERROR_LOGGER_NAME", "dagster.redacted_errors" | ||
) | ||
|
||
error_id_by_exception: ContextVar[Mapping[int, str]] = ContextVar( | ||
"error_id_by_exception", default={} | ||
) | ||
|
||
|
||
@contextlib.contextmanager | ||
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: | ||
try: | ||
yield | ||
except BaseException as e: | ||
exc_info = sys.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()) | ||
|
||
# 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}) | ||
Comment on lines
+100
to
+128
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the way you are using the context var here is equivalent to just having a process global dict. What exactly is the intention here and do any of the existing tests validate that? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i believe the goal is to increase the set of cases that this logic can handle (all kinds of exceptions besides DagsterUserCodeExecutionError can be emitted within user code, like KeyboardInterrupt or SystemExit or other DagsterError subclasses) while still only triggering the redaction if the exception was actually raised within a op_execution_error_boundary or user_code_error_boundary. I don't have a strong opinion about global dict vs. contextvar |
||
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, | ||
) | ||
|
||
# Redact the stacktrace to ensure it will not be passed to Dagster Plus | ||
raise e.with_traceback(None) from None | ||
|
||
|
||
def serializable_error_info_from_exc_info( | ||
exc_info: ExceptionInfo, | ||
|
@@ -116,27 +159,37 @@ 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 | ||
|
||
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, | ||
) | ||
return SerializableErrorInfo( | ||
message=( | ||
f"Error occurred during user code execution, error ID {error_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, | ||
) | ||
from dagster._core.errors import DagsterUserCodeProcessError | ||
|
||
err_id = error_id_by_exception.get().get(id(e)) | ||
if err_id: | ||
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) | ||
Comment on lines
+166
to
+179
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe worth explaining the difference between these two cases - with user code errors, you don't even want to show the message - but with other errors (framework errors or interrupts or Failure / RetryRequested raised within the error boundary), the message is not sensitive and can be displayed for clarity, but the traceback is. |
||
error_info = _serializable_error_info_from_tb(tb_exc) | ||
return SerializableErrorInfo( | ||
message=error_info.message | ||
+ ( | ||
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." | ||
), | ||
stack=[], | ||
cls_name=error_info.cls_name, | ||
cause=None, | ||
context=None, | ||
) | ||
|
||
if ( | ||
hoist_user_code_error | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the name here should probably have 'redacted' in it: redacted_user_code_error_id_by_exception?