* Add a flag for whether an object has been created before
* Add regression test
* doc
* Share object directory between object and node managers
* Treat evicted actor tasks as failed
* minor
* Check return value
* Fix bug where object locations weren't getting updated on client death
* Fix mac build
* Use RayTaskError
ray.wait depends on callbacks from the GCS to decide when an object has appeared in the cluster. The raylet crashes if a callback is received for a wait request that has already completed, but this actually can happen, depending on the order of calls. More precisely:
1. Objects A and B are put in the cluster.
2. Client calls ray.wait([A, B], num_returns=1).
3. Client subscribes to locations for A and B. Locations are cached for both, so callbacks are posted for each.
4. Callback for A fires. The wait completes and the request is removed.
5. Callback for B fires. The wait request no longer exists and raylet crashes.
* Broadcast actor death, clean up dummy objects
* Reduce logging and clean up state when failing a task
* lint
* Make actor failure test nicer, reduce node timeout
* Suppress duplicate pre-emptive object pushes.
* Add test.
* Fix linting
* Remove timer and inline recent_pushes_ into local_objects_.
* Improve test.
* Fix
* Fix linting
* Enable retrying pull from same object manager. Randomize object manager.
* Speed up test
* Linting
* Add test.
* Minor
* Lengthen pull timeout and reissue pull every time a new object becomes available.
* Increase pull timeout in test.
* Wait for nodes to start in object manager test.
* Wait longer for nodes to start up in test.
* Small fixes.
* _submit -> _remote
* Change assert to warning.
* Make scheduling queues RemoveTasks return task states as well.
* Add test
* Don't unsubscribe for infeasible tasks when spilling over.
* Linting
* Address comments.
* speed up task dispatch
* minor changes
* improved comments
* improved comments
* change argument of DispatchTasks to list of tasks
* dispatch only tasks whose dependencies have been fullfiled
* some updated comments
* refactored DispatchQueue() and Assigntask() to avoid the copy of the ready list
* minor fixes
* some more minor fixes
* some more minor fixes
* added more comments
* better comments?
* fixed all feedback comments, minus making the argument of AssignTask() const
* Assigntask() now taskes a const argument
* Do the task copy outside of the callback
* fix linting
## What do these changes do?
This PR exposes the CL option for using a config parameter. This is important for certain tests (i.e., FT tests that removing nodes) to run quickly.
Note that this is bad practice and should be replaced with GFLAGS or some equivalent as soon as possible.
#3239 depends on this.
TODO:
- [x] Add documentation to method arguments before merging.
- [x] Add test to verify this works?
## Related issue number
* Increase timeout to 10s
* Skip eviction reconstruction tests
* Add stress test for many actors to one
* Fix test by shortening it.
* lower number of processes in stress test
* Skip slow test
* Policy that flushes the lineage stash immediately
* Fix bug where remote tasks in uncommitted lineage weren't getting subscribed to, add reg test
* test
* Fix bug where waiting task was getting subscribed
* Cleanup
* Update src/ray/raylet/lineage_cache.cc
Co-Authored-By: stephanie-wang <swang@cs.berkeley.edu>
* Update src/ray/raylet/lineage_cache.cc
Co-Authored-By: stephanie-wang <swang@cs.berkeley.edu>
* cleanup
* cleanup
* Add another test for task with many parents
* fix, unsubscribe to new waiting tasks
* Unsubscribe as soon as the commit notification is handled
We found that there are large amount of pub-sub keys with no content in it (This case is worse when wait-id is used in the key name.).
This logic of deleting empty pub-sub keys from GCS was in legacy ray but not in raylet.
This is fixing a problem that @devin-petersohn observed on the windows subsystem for linux.
In theory, redis should be up once the async connect is happening and there should be no retries needed for the async connect. However on the windows subsystem for linux, the async connect was failing even though the synchronous one was working. Maybe windows has a different semantics here than linux.
This tests the case in which a worker is blocked in a call to ray.get or ray.wait, and then the worker dies. Then later, the object that the worker was waiting for becomes available. We need to make sure not to try to send a message to the dead worker and then die. Related to #2790.