Commit graph

70 commits

Author SHA1 Message Date
Eric Liang
0e00533ed4
Different approach to removing RayGetError (#3471) 2018-12-12 20:30:51 -08:00
Philipp Moritz
06f6431765 Make test_actor_multiple_gpus_from_multiple_tasks less stressful in travis 2018-12-04 17:44:33 -08:00
Stephanie Wang
6b3236349c
Fix memory leak in lineage cache (#3366)
* Move children_ map inside Lineage

* Update lineage_cache.cc

* Test and fixes

* Remove unused
2018-11-21 16:18:39 -08:00
Stephanie Wang
3e33f6f71b
Fix failure handling for actor death (#3359)
* 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
2018-11-21 12:26:22 -08:00
Philipp Moritz
d3697ce4e1
Ready queue refactor to make Dispatching tasks more efficient (#3324)
* put queues outside

* working version, still needs to be optimized

* implement round robin

* proper round robin

* fix spillback

* update

* fix

* cleanup

* more cleanups

* fix

* fix

* add documentation

* explanation for hash combiner

* speed it up

* cleanup and linting

* linting

* comments

* Update scheduling_queue.h

* temp commit

* fixes

* update

* fix

* cleanup

* cleanup

* lint

* more prints

* more prints

* increase sleep

* documentation

* sleep

* fix

* fix

* sleep longer

* update

* fix

* fix

* fix

* Add ordered_set container.

* Fix

* Linting

* Constructors

* Remove O(n) call to list.size().

* fixes

* use ordered set

* Fix.

* Add documentation.

* Add iterators to ordered_set container implementation.

* iterator_type -> iterator

* Make typedefs private

* Add const_iterator

* fix

* fix test

* linting

* lint

* update

* add documentation

* linting
2018-11-20 13:14:12 -08:00
Ujval Misra
b0bfd104f2 Batch heartbeats from node manager together in the monitor. (#3011) 2018-11-20 09:52:27 -08:00
Stephanie Wang
bf88aa5013
Increase timeout before reconstruction is triggered (#3217)
* 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
2018-11-05 18:03:50 -08:00
Robert Nishihara
32f0d6b77e Deprecate num_workers argument to ray.init and ray start. (#3114)
* Remove num_workers argument.

* Fix

* Fix
2018-10-28 20:12:49 -07:00
Robert Nishihara
658c14282c Remove legacy Ray code. (#3121)
* Remove legacy Ray code.

* Fix cmake and simplify monitor.

* Fix linting

* Updates

* Fix

* Implement some methods.

* Remove more plasma manager references.

* Fix

* Linting

* Fix

* Fix

* Make sure class IDs are strings.

* Some path fixes

* Fix

* Path fixes and update arrow

* Fixes.

* linting

* Fixes

* Java fixes

* Some java fixes

* TaskLanguage -> Language

* Minor

* Fix python test and remove unused method signature.

* Fix java tests

* Fix jenkins tests

* Remove commented out code.
2018-10-26 13:36:58 -07:00
Robert Nishihara
9c1826ed69 Use XRay backend by default. (#3020)
* Use XRay backend by default.

* Remove irrelevant valgrind tests.

* Fix

* Move tests around.

* Fix

* Fix test

* Fix test.

* String/unicode fix.

* Fix test

* Fix unicode issue.

* Minor changes

* Fix bug in test_global_state.py.

* Fix test.

* Linting

* Try arrow change and other object manager changes.

* Use newer plasma client API

* Small updates

* Revert plasma client api change.

* Update

* Update arrow and allow SendObjectHeaders to fail.

* Update arrow

* Update python/ray/experimental/state.py

Co-Authored-By: robertnishihara <robertnishihara@gmail.com>

* Address comments.
2018-10-23 12:46:39 -07:00
Philipp Moritz
2c52d9dfa0 Fix actor handle id creation when actor handle was pickled (#3074) 2018-10-17 18:00:52 -07:00
Eric Liang
611259b2c7 Re-raise actor initialization errors on method invocation (#2843)
If an actor constructor fails, save that error and re-raise it on any subsequent attempts to interact with the actor. Related to https://github.com/ray-project/ray/issues/282 and https://github.com/ray-project/ray/issues/1093.
2018-09-10 10:51:19 -07:00
Robert Nishihara
eda6ebb87d Convert some unittests to pytest. (#2779)
* Convert multi_node_test.py to pytest.

* Convert array_test.py to pytest.

* Convert failure_test.py to pytest.

* Convert microbenchmarks to pytest.

* Convert component_failures_test.py to pytest and some minor quotes changes.

* Convert tensorflow_test.py to pytest.

* Convert actor_test.py to pytest.

* Fix.

* Fix
2018-08-31 11:24:15 -07:00
Robert Nishihara
32f7d6fcf5 Add back some tests for xray. (#2772) 2018-08-30 11:07:23 -07:00
Robert Nishihara
132f133214 Limit number of concurrent workers started by hardware concurrency. (#2753)
* Limit number of concurrent workers started by hardware concurrency.

* Check if std:🧵:hardware_concurrency() returns 0.

* Pass in max concurrency from Python.

* Fix Java call to startRaylet.

* Fix typo

* Remove unnecessary cast.

* Fix linting.

* Cleanups on Java side.

* Comment back in actor test.

* Require maximum_startup_concurrency to be at least 1.

* Fix linting and test.

* Improve documentation.

* Fix typo.
2018-08-29 14:53:40 +08:00
Robert Nishihara
b7722897b4 Deprecate 'driver_mode' argument. (#2758)
* Deprecate 'driver_mode' argument.

* Fix

* Fix
2018-08-28 16:45:49 -07:00
Alexey Tumanov
de047daea7 [xray] raylet scheduling mechanism with a simple spillback policy (#2749)
## What do these changes do?
* distribute load and resource information on a heartbeat
* for each raylet, maintain total and available resource capacity as well as measure of current load
* this PR introduces a new notion of load, defined as a sum of all resource demand induced by queued ready tasks on the local raylet. This provides a heterogeneity-aware measure of load that supersedes legacy Ray's task count as a proxy for load.
* modify the scheduling policy to perform *capacity-based*, *load-aware*, *optimistically concurrent* resource allocation
* perform task spillover to the heartbeating node in response to a heartbeat, implementing  heterogeneity-aware late-binding/work-stealing.
2018-08-28 00:03:34 -07:00
Robert Nishihara
aaf5456b3d Add test that tasks sent to actor on dead node raise exceptions. (#2626)
* Add actor failure test.

* Minor change.

* Make test harder.

* Change numbers a bit.

* Skip test for non xray.
2018-08-16 22:48:31 -07:00
Philipp Moritz
d8ba667175 Convert asserts in unittest to pytest (#2529) 2018-08-01 22:32:10 -07:00
Philipp Moritz
696a229ece Fix text verbosity in python 2.7 by running tests with pytest (#2470) 2018-07-30 11:04:06 -07:00
Robert Nishihara
515da7721a Change ray.worker.cleanup -> ray.shutdown and improve API documentation. (#2374)
* Change ray.worker.cleanup -> ray.shutdown and improve API documentation.

* Deprecate ray.worker.cleanup() gracefully.

* Fix linting
2018-07-12 12:00:00 -07:00
Robert Nishihara
54487b1d7f Pin the number of CPUs in failing actor test. (#2368)
* Pin the number of CPUs in failing actor test.

* Pin number of CPUs in multi_node_test.py.

* Fix linting.
2018-07-11 18:34:19 -07:00
Robert Nishihara
18ee044f03 Re-enable some actor tests. (#2276) 2018-06-20 14:42:35 -07:00
Robert Nishihara
61139e1509 Enable fractional resources and resource IDs for xray. (#2187)
* Implement GPU IDs and fractional resources.

* Add documentation and python exceptions.

* Fix signed/unsigned comparison.

* Fix linting.

* Fixes from rebase.

* Re-enable tests that use ray.wait.

* Don't kill the raylet if an infeasible task is submitted.

* Ignore tests that require better load balancing.

* Linting

* Ignore array test.

* Ignore stress test reconstructions tests.

* Don't kill node manager if remote node manager disconnects.

* Ignore more stress tests.

* Naming changes

* Remove outdated todo

* Small fix

* Re-enable test.

* Linting

* Fix resource bookkeeping for blocked tasks.

* Fix linting

* Fix Java client.

* Ignore test

* Ignore put error tests
2018-06-10 15:31:43 -07:00
Robert Nishihara
125fe1c09c Print warning when defining very large remote function or actor. (#2179)
* Print warning when defining very large remote function or actor.

* Add weak test.

* Check that warnings appear in test.

* Make wait_for_errors actually fail in failure_test.py.

* Use constants for error types.

* Fix
2018-06-09 19:59:15 -07:00
Eric Liang
bc2a83e698 Fix support for actor classmethods (#2146) 2018-05-28 17:43:23 -07:00
Yucong He
3509a33cf3 Prototype named actors. (#2129) 2018-05-24 00:32:12 -07:00
Alok Singh
f795173b51 Use flake8-comprehensions (#1976)
* Add flake8 to Travis

* Add flake8-comprehensions

[flake8 plugin](https://github.com/adamchainz/flake8-comprehensions) that
checks for useless constructions.

* Use generators instead of lists where appropriate

A lot of the builtins can take in generators instead of lists.

This commit applies `flake8-comprehensions` to find them.

* Fix lint error

* Fix some string formatting

The rest can be fixed in another PR

* Fix compound literals syntax

This should probably be merged after #1963.

* dict() -> {}

* Use dict literal syntax

dict(...) -> {...}

* Rewrite nested dicts

* Fix hanging indent

* Add missing import

* Add missing quote

* fmt

* Add missing whitespace

* rm duplicate pip install

This is already installed in another file.

* Fix indent

* move `merge_dicts` into utils

* Bring up to date with `master`

* Add automatic syntax upgrade

* rm pyupgrade

In case users want to still use it on their own, the upgrade-syn.sh script was
left in the `.travis` dir.
2018-05-20 16:15:06 -07:00
Adam Gleave
470887c2ad Support calling positional arguments by keyword (fix #998) (#2081) 2018-05-17 16:10:26 -07:00
Melih Elibol
bea97b425b Fix python linting (#2076) 2018-05-16 15:04:31 -07:00
Robert Nishihara
52b0f3734a [xray] Add Travis build for testing xray on Linux. (#2047)
* Run xray tests in travis.

* Comment out TaskTests.testSubmittingManyTasks.

* Comment out failing tests.

* Comment out hanging test.

* Linting

* Comment out failing test.

* Comment out failing test.

* Ignore test_dataframe.py for now.

* Comment out testDriverExitingQuickly.
2018-05-13 21:22:01 -07:00
Robert Nishihara
77c8aa7627 Make ActorHandles pickleable, also make proper ActorHandle and ActorC… (#2007)
* Make ActorHandles pickleable, also make proper ActorHandle and ActorClass classes.

* Fix bug.

* Fix actor test bug.

* Update __ray_terminate__ usage.

* Fix most linting, add documentation, and small cleanups.

* Handle forking and pickling differently for actor handles. Fix linting.

* Fixes for named actors via pickling.

* Generate actor handle IDs deterministically in the pickling case.
2018-05-08 19:19:07 -07:00
Alok Singh
cdf94c18a4 Clean up syntax for supported Python versions. (#1963)
* Use set/dict literal syntax

Ran code through [pyupgrade](https://github.com/asottile/pyupgrade). This is
supported in every Python version 2.7+.

* Drop unnecessary string format specification

No need to specify 0,1.. if paramters are passed in order.

* Revert "Drop unnecessary string format specification"

This reverts commit efa5ec85d30ff69f34e5ed93e31343fea7647bcb.

* Undo changes to cloudpickle

Drop use of set literal until cloudpickle uses it.

* Reformat code with YAPF

We need to set up a git pre-push hook to automatically run this stuff.
2018-05-03 07:45:11 -07:00
Philipp Moritz
74162d1492 Lint Python files with Yapf (#1872) 2018-04-11 10:11:35 -07:00
Robert Nishihara
0c835a379f Fix resource bookkeeping for blocked actor methods. (#1766) 2018-03-21 20:48:04 -07:00
Robert Nishihara
96913be939 Treat actor creation like a regular task. (#1668)
* Treat actor creation like a regular task.

* Small cleanups.

* Change semantics of actor resource handling.

* Bug fix.

* Minor linting

* Bug fix

* Fix jenkins test.

* Fix actor tests

* Some cleanups

* Bug fix

* Fix bug.

* Remove cached actor tasks when a driver is removed.

* Add more info to taskspec in global state API.

* Fix cyclic import bug in tune.

* Fix

* Fix linting.

* Fix linting.

* Don't schedule any tasks (especially actor creaiton tasks) on local schedulers with 0 CPUs.

* Bug fix.

* Add test for 0 CPU case

* Fix linting

* Address comments.

* Fix typos and add comment.

* Add assertion and fix test.
2018-03-16 11:18:07 -07:00
Stephanie Wang
ff8e7f8259
Actor checkpointing for distributed actor handles (#1498)
* Expose calls to get and set the actor frontier

* Remove fields used for old checkpointing prototype, change actor_checkpoint_failed -> succeeded

* Prototype for actor checkpointing

* Filter out duplicate tasks on the local scheduler

* Clean up some of the Python checkpointing code

* More cleanups

* Documentation

* cleanup and fix unit test

* Allow remote checkpoint calls through actor handle

* Check whether object is local before reconstructing

* Enable checkpointing for distributed actor handles, refactor tests

* Fix local scheduler tests

* lint

* Address comments

* lint

* Skip tests that fail on new GCS

* style

* Don't put same object twice when setting the actor frontier

* Address Philipp's comments, cleaner fbs naming
2018-02-07 11:19:32 -08:00
Philipp Moritz
a3f8fa426b Start integrating new GCS APIs (#1379)
* Start integrating new GCS calls

* fixes

* tests

* cleanup

* cleanup and valgrind fix

* update tests

* fix valgrind

* fix more valgrind

* fixes

* add separate tests for GCS

* fix linting

* update tests

* cleanup

* fix python linting

* more fixes

* fix linting

* add plasma manager callback

* add some documentation

* fix linting

* fix linting

* fixes

* update

* fix linting

* fix

* add spillback count

* fixes

* linting

* fixes

* fix linting

* fix

* fix

* fix
2018-01-31 11:01:12 -08:00
Stephanie Wang
74718efa73
Nondeterministic reconstruction for actors (#1344)
* Add failing unit test for nondeterministic reconstruction

* Retry scheduling actor tasks if reassigned to local scheduler

* Update execution edges asynchronously upon dispatch for nondeterministic reconstruction

* Fix bug for updating checkpoint task execution dependencies

* Update comments for deterministic reconstruction

* cleanup

* Add (and skip) failing test case for nondeterministic reconstruction

* Suppress test output
2018-01-21 13:44:13 -08:00
Robert Nishihara
96463c680c Allow actor methods to return multiple object IDs. (#1296)
* Allow actor methods to return multiple object IDs.

* Add test.

* Fixes

* Remove outdated comment.

* Add comment and assert
2017-12-09 10:37:57 -08:00
Robert Nishihara
c21e189371 Allow scheduling with arbitrary user-defined resource labels. (#1236)
* Enable scheduling with custom resource labels.

* Fix.

* Minor fixes and ref counting fix.

* Linting

* Use .data() instead of .c_str().

* Fix linting.

* Fix ResourcesTest.testGPUIDs test by waiting for workers to start up.

* Sleep in test so that all tasks are submitted before any completes.
2017-12-01 11:41:40 -08:00
Robert Nishihara
477a40f76d Prohibit returning actor handles and also update actor documentation. (#1246)
* Prohibit returning actor handles and also update actor documentation.

* Clarify documentation.
2017-11-23 09:37:24 -08:00
Stephanie Wang
af47737bd5 Prototype distributed actor handles (#1137)
* Add actor handle ID to the task spec

* Local scheduler dispatches actor tasks according to a task counter per handle

* Fix python test

* Allow passing actor handles into tasks. Not completely working yet. Also this is very messy.

* Fixes, should be roughly working now.

* Refactor actor handle wrapper

* Fix __init__ tests

* Terminate actor when the original handle goes out of scope

* TODO and a couple test cases

* Make tests for unsupported cases

* Fix Python mode tests

* Linting.

* Cache actor definitions that occur before ray.init() is called.

* Fix export actor class

* Deterministically compute actor handle ID

* Fix __getattribute__

* Fix string encoding for python3

* doc

* Add comment and assertion.
2017-10-19 23:49:59 -07:00
Stephanie Wang
15486a14a0 Refactor actor task queues (#1118)
* Refactor add_task_to_actor_queue into queue_actor_task and insert_actor_task_queue

* Refactor actor task queue to share the waiting task queue

* Fix
2017-10-13 20:52:11 -07:00
Stephanie Wang
3764f2f2e1 Actor checkpointing with object lineage reconstruction (#1004)
* Worker reports error in previous task, actor task counter is incremented after task is successful

* Refactor actor task execution

- Return new task counter in GetTaskRequest
- Update worker state for actor tasks inside of the actor method
  executor

* Manually invoked checkpoint method

* Scheduling for actor checkpoint methods

* Fix python bugs in checkpointing

* Return task success from worker to local scheduler instead of actor counter

* Kill local schedulers halfway through actor execution instead of waiting for all tasks to execute once

* Remove redundant actor tasks during dispatch, reconstruct missing dependencies for actor tasks

* Make executor for temporary actor methods

* doc

* Set default argument for whether the previous task was a success

* Refactor actor method call

* Simplify checkpoint task submission

* lint

* fix philipp's comments

* Add missing line

* Make actor reconstruction tests run faster

* Unimportant whitespace.

* Unimportant whitespace.

* Update checkpoint method signature

* Documentation and handle exceptions during checkpoint save/resume

* Rename get_task message field to actor_checkpoint_failed

* Fix bug.

* Remove debugging check, redirect test output
2017-10-12 09:53:32 -07:00
Robert Nishihara
7a954f4b5f Use monotonic clock for some python tests. (#1112) 2017-10-11 19:58:59 -07:00
Robert Nishihara
4669c59fa8 Release GPU resources as soon as an actor exits. (#1088)
* Release GPU resources as soon as an actor exits.

* Add a test.

* Store local_scheduler_id and driver_id in the worker object instead of the actor object.
2017-10-06 17:58:19 -07:00
Stephanie Wang
aebe9f9374 Fix actor garbage collection by breaking cyclic references (#1064)
* Fix bug in wait_for_pid_to_exit, add test for actor deletion.

* Fix actor garbage collection by breaking cyclic references

* Add test for calling actor method immediately after actor creation.

* Fix bug, must dispatch tasks when workers are killed.

* Fix python test

* Fix cyclic reference problem by creating ActorMethod objects on the fly.

* Try simply increasing the time allowed for many_drivers_test.py.
2017-10-05 00:55:33 -07:00
Stephanie Wang
99c8b1f38c Actor fault tolerance using object lineage reconstruction (#902)
* Revert Python actor reconstruction

* Actor reconstruction using object lineage

* Add dummy arguments and return values for actor tasks

* Pin dummy outputs for actor tasks

* Skip checkpointing test for now

* TODOs

* minor edits

* Generate dummy object dependencies in Python, not C

* Fix linting.

* Move actor counter and dummy objects inside of the actor handle

* Refactor Worker._process_task, suppress exception propagation for
sequential actor tasks
2017-09-10 19:29:28 -07:00
Robert Nishihara
dbe3d9351c Prototype actor checkpointing. (#814)
* Initial testing of checkpointing functions.

* Save checkpoints in Redis.

* Pipe checkpoint_interval through remote decorator.

* Add a test.

* Small cleanups.

* Submit dummy tasks when reconstructing tasks before the most recent tasks so that we don't end up reconstructing the arguments for those tasks.

* Remove old checkpoints to save space.

* Fix linting.
2017-08-07 17:52:39 -07:00