[Data] Fix DataContext deserialization issue with StatsActor#59471
[Data] Fix DataContext deserialization issue with StatsActor#59471raulchen merged 1 commit intoray-project:masterfrom
Conversation
ad3195f to
61a360c
Compare
There was a problem hiding this comment.
Code Review
This pull request effectively addresses a deserialization issue with StatsActor when DataContext contains custom classes from different modules. The introduction of DataContextMetadata to wrap a sanitized version of the DataContext is a clean and robust solution. The sanitization is correctly applied before the data is sent to the StatsActor, preventing ModuleNotFoundError on deserialization. The new test case, test_data_context_with_custom_classes_serialization, is excellent as it accurately reproduces the cross-job scenario and validates the fix. The updates to existing tests to accommodate the new DataContextMetadata are also correctly implemented. I have one minor suggestion to improve resource management in the new test.
Signed-off-by: dragongu <andrewgu@vip.qq.com>
61a360c to
be231b2
Compare
## Description
This PR fixes a deserialization issue in `register_dataset` where
`StatsActor` fails to deserialize `DataContext` parameters containing
custom classes, causing dataset stats management to fail and preventing
Ray Data overview from displaying in the dashboard.
**Problem:**
In a shared Ray cluster with multiple sequential Ray Data jobs:
- The first job runs successfully and creates the global `StatsActor`
- The second job (with different code) runs normally (dataset creation
works), but when its `DataContext` (containing custom classes from
different modules) is serialized to `StatsActor`, the actor fails to
deserialize it
- `StatsActor` logs errors and dashboard cannot display Ray Data
overview because `register_dataset` fails silently
```
ModuleNotFoundError: No module named 'ray_data_utils'
Traceback (most recent call last):
File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 526, in deserialize_objects
obj = self._deserialize_object(...)
File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 363, in _deserialize_object
return self._deserialize_msgpack_data(...)
File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 310, in _deserialize_msgpack_data
python_objects = self._deserialize_pickle5_data(...)
File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 292, in _deserialize_pickle5_data
obj = pickle.loads(in_band)
ModuleNotFoundError: No module named 'ray_data_utils'
```
This error occurs because `DataContext` contains objects (e.g., custom
classes) that reference modules not available in `StatsActor`'s runtime
environment. Similar errors can occur for other custom classes, such as
`ModuleNotFoundError: No module named 'test_custom_module'` when custom
exception classes are used.
**Reproduction Script:**
To reproduce the issue before the fix, you can use the following script:
```python
#!/usr/bin/env python3
"""Reproduction script for DataContext serialization issue."""
import os
import tempfile
import ray
import ray.data
from ray._private.test_utils import run_string_as_driver
def create_driver_script_with_dependency(working_dir, ray_address):
"""Create custom module and driver script that depends on it."""
custom_module_path = os.path.join(working_dir, "test_custom_module.py")
with open(custom_module_path, "w") as f:
f.write(
"""class CustomRetryException(Exception):
def __init__(self):
pass
"""
)
driver_script = f"""
import sys
import os
os.chdir(r"{working_dir}")
import ray
import ray.data
from ray.data.context import DataContext
ray.init(
address="{ray_address}",
ignore_reinit_error=True,
runtime_env={{"working_dir": r"{working_dir}"}}
)
import test_custom_module
data_context = DataContext.get_current()
data_context.actor_task_retry_on_errors = [test_custom_module.CustomRetryException]
ds = ray.data.range(10)
ds.take(1)
ray.shutdown()
"""
return driver_script
if __name__ == "__main__":
ray.init()
# Job 1: Create dataset to trigger StatsActor creation
ds = ray.data.range(10)
ds.take(1)
# Job 2: Run job that imports custom exception from module
working_dir = os.path.abspath(tempfile.mkdtemp())
ray_address = ray.get_runtime_context().gcs_address
driver_script = create_driver_script_with_dependency(working_dir, ray_address)
run_string_as_driver(driver_script)
```
**Root Cause:**
`StatsActor` is a global actor that persists across jobs. When a job
submits `DataContext` containing custom classes, `cloudpickle` tries to
deserialize them, but the custom class's module may not be in
`StatsActor`'s `PYTHONPATH`, causing `ModuleNotFoundError`.
**Solution:**
Sanitize `DataContext` before serialization using
`sanitize_for_struct()` to convert custom classes to basic types (dicts,
lists, strings, numbers), avoiding module dependency issues. Introduced
`DataContextMetadata` to wrap the sanitized configuration.
## Related issues
Fixes deserialization failures when `DataContext` contains custom
exception classes from project modules.
## Testing
- Added unit test `test_data_context_with_custom_classes_serialization`
that:
- Creates a first job to trigger `StatsActor` creation
- Submits a second job that imports a custom exception from a module
- Sets the custom exception in `DataContext.actor_task_retry_on_errors`
- Verifies that the dataset registration succeeds without
`ModuleNotFoundError`
- Confirms `StatsActor` can retrieve datasets without errors
The test reproduces the real-world scenario where different jobs submit
`DataContext` with custom classes to a shared `StatsActor`.
## Additional Context
This fix resolves deserialization issues for all custom classes in
`DataContext`, including:
- Custom exception classes in `actor_task_retry_on_errors` (e.g.,
`test_custom_module.CustomRetryException`)
- Objects with `__module__` pointing to non-existent modules (e.g.,
`ray_data_utils`)
- Any other custom classes that reference modules not available in
`StatsActor`'s runtime environment
By sanitizing `DataContext` before serialization using
`sanitize_for_struct()`, all custom classes are converted to dictionary
representations, eliminating module dependency issues entirely.
Signed-off-by: dragongu <andrewgu@vip.qq.com>
…ject#59471) ## Description This PR fixes a deserialization issue in `register_dataset` where `StatsActor` fails to deserialize `DataContext` parameters containing custom classes, causing dataset stats management to fail and preventing Ray Data overview from displaying in the dashboard. **Problem:** In a shared Ray cluster with multiple sequential Ray Data jobs: - The first job runs successfully and creates the global `StatsActor` - The second job (with different code) runs normally (dataset creation works), but when its `DataContext` (containing custom classes from different modules) is serialized to `StatsActor`, the actor fails to deserialize it - `StatsActor` logs errors and dashboard cannot display Ray Data overview because `register_dataset` fails silently ``` ModuleNotFoundError: No module named 'ray_data_utils' Traceback (most recent call last): File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 526, in deserialize_objects obj = self._deserialize_object(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 363, in _deserialize_object return self._deserialize_msgpack_data(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 310, in _deserialize_msgpack_data python_objects = self._deserialize_pickle5_data(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 292, in _deserialize_pickle5_data obj = pickle.loads(in_band) ModuleNotFoundError: No module named 'ray_data_utils' ``` This error occurs because `DataContext` contains objects (e.g., custom classes) that reference modules not available in `StatsActor`'s runtime environment. Similar errors can occur for other custom classes, such as `ModuleNotFoundError: No module named 'test_custom_module'` when custom exception classes are used. **Reproduction Script:** To reproduce the issue before the fix, you can use the following script: ```python #!/usr/bin/env python3 """Reproduction script for DataContext serialization issue.""" import os import tempfile import ray import ray.data from ray._private.test_utils import run_string_as_driver def create_driver_script_with_dependency(working_dir, ray_address): """Create custom module and driver script that depends on it.""" custom_module_path = os.path.join(working_dir, "test_custom_module.py") with open(custom_module_path, "w") as f: f.write( """class CustomRetryException(Exception): def __init__(self): pass """ ) driver_script = f""" import sys import os os.chdir(r"{working_dir}") import ray import ray.data from ray.data.context import DataContext ray.init( address="{ray_address}", ignore_reinit_error=True, runtime_env={{"working_dir": r"{working_dir}"}} ) import test_custom_module data_context = DataContext.get_current() data_context.actor_task_retry_on_errors = [test_custom_module.CustomRetryException] ds = ray.data.range(10) ds.take(1) ray.shutdown() """ return driver_script if __name__ == "__main__": ray.init() # Job 1: Create dataset to trigger StatsActor creation ds = ray.data.range(10) ds.take(1) # Job 2: Run job that imports custom exception from module working_dir = os.path.abspath(tempfile.mkdtemp()) ray_address = ray.get_runtime_context().gcs_address driver_script = create_driver_script_with_dependency(working_dir, ray_address) run_string_as_driver(driver_script) ``` **Root Cause:** `StatsActor` is a global actor that persists across jobs. When a job submits `DataContext` containing custom classes, `cloudpickle` tries to deserialize them, but the custom class's module may not be in `StatsActor`'s `PYTHONPATH`, causing `ModuleNotFoundError`. **Solution:** Sanitize `DataContext` before serialization using `sanitize_for_struct()` to convert custom classes to basic types (dicts, lists, strings, numbers), avoiding module dependency issues. Introduced `DataContextMetadata` to wrap the sanitized configuration. ## Related issues Fixes deserialization failures when `DataContext` contains custom exception classes from project modules. ## Testing - Added unit test `test_data_context_with_custom_classes_serialization` that: - Creates a first job to trigger `StatsActor` creation - Submits a second job that imports a custom exception from a module - Sets the custom exception in `DataContext.actor_task_retry_on_errors` - Verifies that the dataset registration succeeds without `ModuleNotFoundError` - Confirms `StatsActor` can retrieve datasets without errors The test reproduces the real-world scenario where different jobs submit `DataContext` with custom classes to a shared `StatsActor`. ## Additional Context This fix resolves deserialization issues for all custom classes in `DataContext`, including: - Custom exception classes in `actor_task_retry_on_errors` (e.g., `test_custom_module.CustomRetryException`) - Objects with `__module__` pointing to non-existent modules (e.g., `ray_data_utils`) - Any other custom classes that reference modules not available in `StatsActor`'s runtime environment By sanitizing `DataContext` before serialization using `sanitize_for_struct()`, all custom classes are converted to dictionary representations, eliminating module dependency issues entirely. Signed-off-by: dragongu <andrewgu@vip.qq.com>
…ject#59471) ## Description This PR fixes a deserialization issue in `register_dataset` where `StatsActor` fails to deserialize `DataContext` parameters containing custom classes, causing dataset stats management to fail and preventing Ray Data overview from displaying in the dashboard. **Problem:** In a shared Ray cluster with multiple sequential Ray Data jobs: - The first job runs successfully and creates the global `StatsActor` - The second job (with different code) runs normally (dataset creation works), but when its `DataContext` (containing custom classes from different modules) is serialized to `StatsActor`, the actor fails to deserialize it - `StatsActor` logs errors and dashboard cannot display Ray Data overview because `register_dataset` fails silently ``` ModuleNotFoundError: No module named 'ray_data_utils' Traceback (most recent call last): File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 526, in deserialize_objects obj = self._deserialize_object(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 363, in _deserialize_object return self._deserialize_msgpack_data(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 310, in _deserialize_msgpack_data python_objects = self._deserialize_pickle5_data(...) File "/usr/local/lib/python3.11/dist-packages/ray/_private/serialization.py", line 292, in _deserialize_pickle5_data obj = pickle.loads(in_band) ModuleNotFoundError: No module named 'ray_data_utils' ``` This error occurs because `DataContext` contains objects (e.g., custom classes) that reference modules not available in `StatsActor`'s runtime environment. Similar errors can occur for other custom classes, such as `ModuleNotFoundError: No module named 'test_custom_module'` when custom exception classes are used. **Reproduction Script:** To reproduce the issue before the fix, you can use the following script: ```python #!/usr/bin/env python3 """Reproduction script for DataContext serialization issue.""" import os import tempfile import ray import ray.data from ray._private.test_utils import run_string_as_driver def create_driver_script_with_dependency(working_dir, ray_address): """Create custom module and driver script that depends on it.""" custom_module_path = os.path.join(working_dir, "test_custom_module.py") with open(custom_module_path, "w") as f: f.write( """class CustomRetryException(Exception): def __init__(self): pass """ ) driver_script = f""" import sys import os os.chdir(r"{working_dir}") import ray import ray.data from ray.data.context import DataContext ray.init( address="{ray_address}", ignore_reinit_error=True, runtime_env={{"working_dir": r"{working_dir}"}} ) import test_custom_module data_context = DataContext.get_current() data_context.actor_task_retry_on_errors = [test_custom_module.CustomRetryException] ds = ray.data.range(10) ds.take(1) ray.shutdown() """ return driver_script if __name__ == "__main__": ray.init() # Job 1: Create dataset to trigger StatsActor creation ds = ray.data.range(10) ds.take(1) # Job 2: Run job that imports custom exception from module working_dir = os.path.abspath(tempfile.mkdtemp()) ray_address = ray.get_runtime_context().gcs_address driver_script = create_driver_script_with_dependency(working_dir, ray_address) run_string_as_driver(driver_script) ``` **Root Cause:** `StatsActor` is a global actor that persists across jobs. When a job submits `DataContext` containing custom classes, `cloudpickle` tries to deserialize them, but the custom class's module may not be in `StatsActor`'s `PYTHONPATH`, causing `ModuleNotFoundError`. **Solution:** Sanitize `DataContext` before serialization using `sanitize_for_struct()` to convert custom classes to basic types (dicts, lists, strings, numbers), avoiding module dependency issues. Introduced `DataContextMetadata` to wrap the sanitized configuration. ## Related issues Fixes deserialization failures when `DataContext` contains custom exception classes from project modules. ## Testing - Added unit test `test_data_context_with_custom_classes_serialization` that: - Creates a first job to trigger `StatsActor` creation - Submits a second job that imports a custom exception from a module - Sets the custom exception in `DataContext.actor_task_retry_on_errors` - Verifies that the dataset registration succeeds without `ModuleNotFoundError` - Confirms `StatsActor` can retrieve datasets without errors The test reproduces the real-world scenario where different jobs submit `DataContext` with custom classes to a shared `StatsActor`. ## Additional Context This fix resolves deserialization issues for all custom classes in `DataContext`, including: - Custom exception classes in `actor_task_retry_on_errors` (e.g., `test_custom_module.CustomRetryException`) - Objects with `__module__` pointing to non-existent modules (e.g., `ray_data_utils`) - Any other custom classes that reference modules not available in `StatsActor`'s runtime environment By sanitizing `DataContext` before serialization using `sanitize_for_struct()`, all custom classes are converted to dictionary representations, eliminating module dependency issues entirely. Signed-off-by: dragongu <andrewgu@vip.qq.com>
Description
This PR fixes a deserialization issue in
register_datasetwhereStatsActorfails to deserializeDataContextparameters containing custom classes, causing dataset stats management to fail and preventing Ray Data overview from displaying in the dashboard.Problem:
In a shared Ray cluster with multiple sequential Ray Data jobs:
StatsActorDataContext(containing custom classes from different modules) is serialized toStatsActor, the actor fails to deserialize itStatsActorlogs errors and dashboard cannot display Ray Data overview becauseregister_datasetfails silentlyThis error occurs because
DataContextcontains objects (e.g., custom classes) that reference modules not available inStatsActor's runtime environment. Similar errors can occur for other custom classes, such asModuleNotFoundError: No module named 'test_custom_module'when custom exception classes are used.Reproduction Script:
To reproduce the issue before the fix, you can use the following script:
Root Cause:
StatsActoris a global actor that persists across jobs. When a job submitsDataContextcontaining custom classes,cloudpickletries to deserialize them, but the custom class's module may not be inStatsActor'sPYTHONPATH, causingModuleNotFoundError.Solution:
Sanitize
DataContextbefore serialization usingsanitize_for_struct()to convert custom classes to basic types (dicts, lists, strings, numbers), avoiding module dependency issues. IntroducedDataContextMetadatato wrap the sanitized configuration.Related issues
Fixes deserialization failures when
DataContextcontains custom exception classes from project modules.Testing
test_data_context_with_custom_classes_serializationthat:StatsActorcreationDataContext.actor_task_retry_on_errorsModuleNotFoundErrorStatsActorcan retrieve datasets without errorsThe test reproduces the real-world scenario where different jobs submit
DataContextwith custom classes to a sharedStatsActor.Additional Context
This fix resolves deserialization issues for all custom classes in
DataContext, including:actor_task_retry_on_errors(e.g.,test_custom_module.CustomRetryException)__module__pointing to non-existent modules (e.g.,ray_data_utils)StatsActor's runtime environmentBy sanitizing
DataContextbefore serialization usingsanitize_for_struct(), all custom classes are converted to dictionary representations, eliminating module dependency issues entirely.