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

Redis LRU eviction causes hangs with ray.wait() (but get() is fine) #3997

Closed
ericl opened this issue Feb 8, 2019 · 13 comments · Fixed by #4021
Closed

Redis LRU eviction causes hangs with ray.wait() (but get() is fine) #3997

ericl opened this issue Feb 8, 2019 · 13 comments · Fixed by #4021
Assignees

Comments

@ericl
Copy link
Contributor

ericl commented Feb 8, 2019

System information

  • Ray version: 0.6.4

Describe the problem

import ray

@ray.remote
def f():
    return 0

@ray.remote
def g():
    import time
    start = time.time()
    while time.time() < start + 1:
        ray.get([f.remote() for _ in range(10)])

import ray
# 10MB -> hangs after ~5 iterations
# 20MB -> hangs after ~20 iterations
# 50MB -> hangs after ~50 iterations
ray.init(redis_max_memory=1024 * 1024 * 50)

i = 0
while True:
    i += 1
    a = g.remote()
    [ok], _ = ray.wait([a])
    print("iter", i)

Source code / logs

The above example will reproducibly hang with number of iterations proportional to the redis memory size.

The expected behaviour is that once the memory size is large enough, it can run forever.

Also, we shouldn't be hanging and should throw an error if redis capacity is too small.

@ericl
Copy link
Contributor Author

ericl commented Feb 8, 2019

cc @stephanie-wang

@ericl ericl changed the title Redis LRU eviction causes hangs with wait on actor tasks Redis LRU eviction causes hangs with ray.wait() Feb 8, 2019
@ericl
Copy link
Contributor Author

ericl commented Feb 8, 2019

Note that ray.get() does not have the same problem.

@ericl ericl changed the title Redis LRU eviction causes hangs with ray.wait() Redis LRU eviction causes hangs with ray.wait() (but get() is fine) Feb 8, 2019
@stephanie-wang stephanie-wang self-assigned this Feb 8, 2019
@stephanie-wang
Copy link
Contributor

Weirdly, I can see the log messages for when the raylet adds the object location to the GCS. I can also see using MONITOR on the redis instance that the GCS adds the key, then calls PUBLISH. However, somehow the GCS client that is waiting for the PUBLISH notification never receives it.

It could be that the key is getting evicted from redis right after the PUBLISH command. I don't really understand what the guarantees are in that case.

@ericl
Copy link
Contributor Author

ericl commented Feb 8, 2019

It seems unlikely that key is getting evicted by LRU, unless the policy is going very wrong (would have to be nearly MRU). What's weird is that you get a hang no matter how large redis is, it just takes longer.

@stephanie-wang
Copy link
Contributor

Hmm yeah, that's true. Also, I did find that I could look up the object table entry in the GCS afterwards.

Could be some bug in how we're requesting notifications.

@ericl
Copy link
Contributor Author

ericl commented Feb 8, 2019

Could it be evicting something else important long-lived (subscription keys) which is not used for get?

@stephanie-wang
Copy link
Contributor

So far I can't figure out how the behavior that I'm observing relates to eviction. When I start another Redis client manually and subscribe to the object table, that client is able to receive the notification that the raylet is somehow not getting.

Also, I think the reason ray.get works is because the Python worker is just polling the object store until the object becomes local. It doesn't block on the GCS (unless it's multinode and has to subscribe to object locations to pull the data).

@stephanie-wang
Copy link
Contributor

Okay, just remembered that object table notifications only get sent to clients who requested notifications for a specific object. And the set of clients to notify is itself stored as a Redis entry. So most likely that key is getting evicted, so the client never receives the notification.

Two options:

  1. Mark those types of entries as unevictable. It would be fine to never evict them since these entries are short-lived anyway.
  2. Store the clients as a C data structure instead of directly in Redis.

@pcmoritz
Copy link
Contributor

pcmoritz commented Feb 9, 2019

Ah, great find :)

It might be a bit hard to mark entries as unevictable (the only way I see is setting expires for all other keys in which case redis will evict them before it evicts keys without expires, but it might make everything slower).

So we should probably just keep track of the client in C++ datastructures.

@ericl
Copy link
Contributor Author

ericl commented Feb 9, 2019

Ah nice! Another possible quick fix is to move that table of clients to notify to the primary redis, is that feasible?

@stephanie-wang
Copy link
Contributor

Ah nice! Another possible quick fix is to move that table of clients to notify to the primary redis, is that feasible?

Yeah, that should work, but it's probably not preferable as a long-term solution since it's often on the critical path for task execution and there may be many active keys (as many as there are objects required).

@robertnishihara
Copy link
Collaborator

@stephanie-wang when there are clients waiting to be notified, shouldn't we also make the corresponding object table key unevictable? Since we know we're going to use it in the future?

@stephanie-wang
Copy link
Contributor

stephanie-wang commented Feb 11, 2019

Never mind, moving the table to the primary shard won't work since that means moving the table that you're subscribing to (the object table in this case).

Hmm, @robertnishihara, we could probably have the redis server touch the relevant key (if it exists) whenever a notification is requested.

Edit: Actually, in many cases i think it will probably just work out without touching the object table key. Usually when a client requests notifications from the object table, it's because the key is currently empty and the client wants to know when the key has a value (i.e. the object has a location). Not sure how this would work out if a client needs notifications for a particular object for a long time, though.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants