Skip to content

Commit 67576ee

Browse files
committed
special case exception types
1 parent 9da71cf commit 67576ee

File tree

2 files changed

+53
-47
lines changed

2 files changed

+53
-47
lines changed

python_modules/dagster/dagster/_utils/error.py

Lines changed: 28 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@
1212
from typing_extensions import TypeAlias
1313

1414
import dagster._check as check
15+
from dagster._core.errors import DagsterUserCodeExecutionError
1516
from dagster._serdes import whitelist_for_serdes
1617

1718

@@ -154,17 +155,33 @@ def serializable_error_info_from_exc_info(
154155

155156
err_id = error_id_by_exception.get().get(id(e))
156157
if err_id:
157-
return SerializableErrorInfo(
158-
message=(
159-
f"Error occurred during user code execution, error ID {err_id}. "
160-
"The error has been masked to prevent leaking sensitive information. "
161-
"Search in logs for this error ID for more details."
162-
),
163-
stack=[],
164-
cls_name="DagsterRedactedUserCodeError",
165-
cause=None,
166-
context=None,
167-
)
158+
if isinstance(e, DagsterUserCodeExecutionError):
159+
return SerializableErrorInfo(
160+
message=(
161+
f"Error occurred during user code execution, error ID {err_id}. "
162+
"The error has been masked to prevent leaking sensitive information. "
163+
"Search in logs for this error ID for more details."
164+
),
165+
stack=[],
166+
cls_name="DagsterRedactedUserCodeError",
167+
cause=None,
168+
context=None,
169+
)
170+
else:
171+
tb_exc = traceback.TracebackException(exc_type, e, tb)
172+
error_info = _serializable_error_info_from_tb(tb_exc)
173+
return SerializableErrorInfo(
174+
message=error_info.message
175+
+ (
176+
"Error ID {err_id}. "
177+
"The error has been masked to prevent leaking sensitive information. "
178+
"Search in logs for this error ID for more details."
179+
),
180+
stack=[],
181+
cls_name=error_info.cls_name,
182+
cause=None,
183+
context=None,
184+
)
168185

169186
if (
170187
hoist_user_code_error

python_modules/dagster/dagster_tests/core_tests/test_mask_user_code_errors.py

Lines changed: 25 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -6,8 +6,10 @@
66

77
import pytest
88
from dagster import Config, RunConfig, config_mapping, job, op
9+
from dagster._core.definitions.events import Failure
910
from dagster._core.definitions.timestamp import TimestampWithTimezone
1011
from dagster._core.errors import (
12+
DagsterExecutionInterruptedError,
1113
DagsterUserCodeExecutionError,
1214
DagsterUserCodeProcessError,
1315
user_code_error_boundary,
@@ -84,34 +86,6 @@ def hunter2():
8486
assert "hunter2" not in str(err_info)
8587

8688

87-
def test_masking_nested_user_code_err_reraise(enable_masking_user_code_errors):
88-
class MyException(Exception):
89-
inner: Exception
90-
91-
def __init__(self, inner: Exception):
92-
self.inner = inner
93-
94-
try:
95-
try:
96-
with user_code_error_boundary(
97-
error_cls=DagsterUserCodeExecutionError,
98-
msg_fn=lambda: "test",
99-
):
100-
101-
def hunter2():
102-
raise UserError()
103-
104-
hunter2()
105-
except Exception as e:
106-
raise MyException(e) from e
107-
108-
except Exception:
109-
exc_info = sys.exc_info()
110-
err_info = serializable_error_info_from_exc_info(exc_info)
111-
112-
assert "hunter2" not in str(err_info)
113-
114-
11589
def test_masking_nested_user_code_err_boundaries_reraise(enable_masking_user_code_errors):
11690
try:
11791
try:
@@ -140,12 +114,20 @@ def hunter2():
140114

141115

142116
@pytest.mark.parametrize(
143-
"build_exc",
144-
[lambda: UserError(), lambda: KeyboardInterrupt("hunter2"), lambda: hunter2() - 5], # type:ignore
145-
ids=["UserError", "KeyboardInterrupt", "TypeError"],
117+
"exc_name, expect_exc_name_in_error, build_exc",
118+
[
119+
("UserError", False, lambda: UserError()),
120+
("TypeError", False, lambda: TypeError("hunter2")),
121+
("KeyboardInterrupt", True, lambda: KeyboardInterrupt()),
122+
("DagsterExecutionInterruptedError", True, lambda: DagsterExecutionInterruptedError()),
123+
("Failure", True, lambda: Failure("asdf")),
124+
],
146125
)
147126
def test_masking_op_execution(
148-
enable_masking_user_code_errors, build_exc: Callable[[], BaseException]
127+
enable_masking_user_code_errors,
128+
exc_name: str,
129+
expect_exc_name_in_error: bool,
130+
build_exc: Callable[[], BaseException],
149131
) -> Any:
150132
@op
151133
def throws_user_error(_):
@@ -167,10 +149,17 @@ def job_def():
167149
]
168150

169151
step_error = next(event for event in result.all_events if event.is_step_failure)
170-
assert (
171-
step_error.step_failure_data.error
172-
and step_error.step_failure_data.error.cls_name == "DagsterRedactedUserCodeError"
173-
)
152+
153+
if expect_exc_name_in_error:
154+
assert (
155+
step_error.step_failure_data.error
156+
and step_error.step_failure_data.error.cls_name == exc_name
157+
)
158+
else:
159+
assert (
160+
step_error.step_failure_data.error
161+
and step_error.step_failure_data.error.cls_name == "DagsterRedactedUserCodeError"
162+
)
174163

175164

176165
ERROR_ID_REGEX = r"Error occurred during user code execution, error ID ([a-z0-9\-]+)"

0 commit comments

Comments
 (0)