-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
Add some sanity checks for memory use in dataset #22642
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
Validating CODEOWNERS rules …
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,10 +1,12 @@ | ||
import pytest | ||
import numpy as np | ||
import pandas as pd | ||
import os | ||
|
||
import ray | ||
from ray.data.context import DatasetContext | ||
from ray.data.datasource.csv_datasource import CSVDatasource | ||
from ray.internal.internal_api import memory_summary | ||
|
||
from ray.tests.conftest import * # noqa | ||
|
||
|
@@ -17,6 +19,70 @@ def expect_stages(pipe, num_stages_expected, stage_names): | |
assert len(pipe._optimized_stages) == num_stages_expected, pipe._optimized_stages | ||
|
||
|
||
def test_memory_sanity(shutdown_only): | ||
info = ray.init(num_cpus=1, object_store_memory=500e6) | ||
ds = ray.data.range(10) | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
meminfo = memory_summary(info.address_info["address"], stats_only=True) | ||
|
||
# Sanity check spilling is happening as expected. | ||
assert "Spilled" in meminfo, meminfo | ||
|
||
|
||
def test_memory_release_eager(shutdown_only): | ||
info = ray.init(num_cpus=1, object_store_memory=1500e6) | ||
ds = ray.data.range(10) | ||
|
||
# Round 1. | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
meminfo = memory_summary(info.address_info["address"], stats_only=True) | ||
assert "Spilled" not in meminfo, meminfo | ||
|
||
# Round 2. | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
meminfo = memory_summary(info["address"], stats_only=True) | ||
# TODO(ekl) we can add this back once we clear inputs on eager exec as well. | ||
# assert "Spilled" not in meminfo, meminfo | ||
|
||
|
||
def test_memory_release_lazy(shutdown_only): | ||
info = ray.init(num_cpus=1, object_store_memory=1500e6) | ||
ds = ray.data.range(10) | ||
|
||
# Should get fused into single stage. | ||
ds = ds._experimental_lazy() | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
ds.fully_executed() | ||
meminfo = memory_summary(info.address_info["address"], stats_only=True) | ||
assert "Spilled" not in meminfo, meminfo | ||
|
||
|
||
def test_memory_release_lazy_shuffle(shutdown_only): | ||
# TODO(ekl) why is this flaky? Due to eviction delay? | ||
error = None | ||
for trial in range(3): | ||
print("Try", trial) | ||
try: | ||
info = ray.init(num_cpus=1, object_store_memory=1800e6) | ||
ds = ray.data.range(10) | ||
|
||
# Should get fused into single stage. | ||
ds = ds._experimental_lazy() | ||
ds = ds.map(lambda x: np.ones(100 * 1024 * 1024, dtype=np.uint8)) | ||
ds.random_shuffle().fully_executed() | ||
meminfo = memory_summary(info.address_info["address"], stats_only=True) | ||
assert "Spilled" not in meminfo, meminfo | ||
return | ||
except Exception as e: | ||
error = e | ||
print("Failed", e) | ||
finally: | ||
ray.shutdown() | ||
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. will this cause double shutdown since you also have shutdown_only? 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. Yea; though it is harmless. |
||
raise error | ||
|
||
|
||
def test_spread_hint_inherit(ray_start_regular_shared): | ||
ds = ray.data.range(10)._experimental_lazy() | ||
ds = ds.map(lambda x: x + 1) | ||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
FYI @stephanie-wang , not sure if you also observed this in your shuffle experiments (memory sometimes not getting released in a timely manner during shuffle)
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.
Hmm I think @franklsf95 actually did see something like this.
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.
It's quite odd. I don't think it's related to Python GC or anything. I figured it might be a unit-test artifact since we have some batching/delay before releasing memory, which would only show up in a fast test, but it could also be an issue at larger scale.