Skip to content
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

[Ray] Fix ray memory leak #3184

Merged
merged 8 commits into from
Aug 3, 2022

Conversation

fyrestone
Copy link
Contributor

@fyrestone fyrestone commented Jul 11, 2022

What do these changes do?

The memory leak is because when starting session by the new_ray_session, the global router is not set. There is a _set_global_router in ActorPoolBase::create, but the new_ray_session calls the create method in Ray actor. So, driver still has empty global router.

The memory leak details:

  1. new_ray_session has not called the Router.set_instance.
  2. Each Mars actor call invokes Router.get_instance_or_empty(), a new Router instance will be created.
  3. The Router instance is always a new one, then router.get_client(...) creates a new client instance in ActorCaller.
  4. The asyncio.create_task(self._listen(client)) creates a new coroutine _listen.
  5. The _listen coroutine captures the returned ResultMessage in frame.

Before

[ Top 10 differences ]
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/serialization.py:160: size=63.6 MiB (+60.5 MiB), count=1678 (+1606), average=38.8 KiB
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/serialization.py:162: size=880 KiB (+815 KiB), count=3727 (+3301), average=242 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/actor.py:947: size=250 KiB (+201 KiB), count=2914 (+2340), average=88 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/actor.py:955: size=244 KiB (+195 KiB), count=225 (+180), average=1109 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/actor.py:101: size=216 KiB (+170 KiB), count=4195 (+3325), average=53 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/asyncio/locks.py:244: size=185 KiB (+153 KiB), count=1203 (+981), average=158 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/actor.py:954: size=187 KiB (+150 KiB), count=2968 (+2386), average=64 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/asyncio/queues.py:51: size=136 KiB (+112 KiB), count=436 (+360), average=319 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/connection.py:225: size=127 KiB (+102 KiB), count=223 (+179), average=584 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/ray/actor.py:949: size=126 KiB (+101 KiB), count=224 (+180), average=576 B

After

[ Top 10 differences ]
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/connection.py:194: size=1000 B (-64.0 KiB), count=14 (+0), average=71 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/connection.py:225: size=24.0 KiB (+8176 B), count=42 (+14), average=584 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/threading.py:552: size=2760 B (-2208 B), count=5 (-4), average=552 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/client.py:1436: size=12.7 KiB (-2032 B), count=69 (-9), average=189 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/client.py:1422: size=6336 B (-1200 B), count=12 (-3), average=528 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/json/decoder.py:353: size=2640 B (+1192 B), count=32 (+14), average=82 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/pandas/core/indexes/base.py:495: size=1752 B (+1168 B), count=3 (+2), average=584 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/threading.py:348: size=7392 B (+1056 B), count=14 (+2), average=528 B
/home/admin/.pyenv/versions/3.7.7/lib/python3.7/site-packages/redis/client.py:1421: size=2672 B (-1040 B), count=11 (-3), average=243 B
<unknown>:0: size=42.1 KiB (+1008 B), count=144 (+6), average=299 

Related issue number

Fixes #3183
Fixes #3173

Check code requirements

  • tests added / passed (if needed)
  • Ensure all linting tests pass, see here for how to run them

@fyrestone fyrestone force-pushed the fix_ray_memory_leak branch from b0a904d to 5fd8c46 Compare July 28, 2022 07:04
刘宝 and others added 2 commits July 28, 2022 17:24
@fyrestone fyrestone marked this pull request as ready for review July 29, 2022 06:41
@fyrestone fyrestone requested a review from a team as a code owner July 29, 2022 06:41
@fyrestone fyrestone added the type: bug Something isn't working label Jul 29, 2022
zhongchun
zhongchun previously approved these changes Aug 1, 2022
Copy link
Contributor

@zhongchun zhongchun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@chaokunyang
Copy link
Contributor

chaokunyang commented Aug 1, 2022

_listen in ActorCaller may leak memory too, can we fix it in this PR?

@fyrestone
Copy link
Contributor Author

_listen in ActorCaller may leak memory too, can we fix it in this PR?

You mean free the local vars (ResultMessage object) in _listen coroutine as soon as possible?

@chaokunyang
Copy link
Contributor

_listen in ActorCaller may leak memory too, can we fix it in this PR?

You mean free the local vars (ResultMessage object) in _listen coroutine as soon as possible?

Yes, local vars (ResultMessage object) in _listen coroutine will leak if client.recv() doesn't return timely.

@fyrestone
Copy link
Contributor Author

_listen in ActorCaller may leak memory too, can we fix it in this PR?

You mean free the local vars (ResultMessage object) in _listen coroutine as soon as possible?

Yes, local vars (ResultMessage object) in _listen coroutine will leak if client.recv() doesn't return timely.

Fixed.

Copy link
Contributor

@chaokunyang chaokunyang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

@zhongchun zhongchun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

Copy link
Contributor

@zhongchun zhongchun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@fyrestone fyrestone merged commit 7417139 into mars-project:master Aug 3, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
mod: ray integration type: bug Something isn't working
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[BUG] new_ray_session() has memory leak in driver What's about python version compatible?
3 participants