Commit graph

94 commits

Author SHA1 Message Date
Melih Elibol
6e06a9e338 XRay Task Forwarding Milestone (#1785)
Summary:
Able to run 1000 tasks with object dependencies on a set of distributed Raylets.

Raylet Changes:

Finalized ClientConnection class.
Task forwarding.
NM-to-NM heartbeats.
NM resource accounting for tasks.
Simple scheduling policy with task forwarding.
Creating and maintaining NM 2 NM long-lived connections and reusing them for task forwarding.
LineageCache Changes:

LineageCache without cleanup of tasks committed by remote nodes.
Lineage cache writeback and cleanup implementation.
ObjectManager Changes:

Object manager event loop/ClientConnection refactor.
Multithreaded object manager (disabled in this PR).
Testing Changes:

Integration tests for task submission on multiple Raylets.
Stress tests for object manager (with GCS and object store integration).


Co-authored-by: Stephanie Wang <swang@cs.berkeley.edu>
Co-authored-by: Alexey Tumanov <atumanov@gmail.com>
2018-03-31 18:02:58 -07:00
Robert Nishihara
0c835a379f Fix resource bookkeeping for blocked actor methods. (#1766) 2018-03-21 20:48:04 -07:00
Robert Nishihara
4bccabd910 Redirect output of all processes by default. (#1752)
* Redirect output of all processes by default.

* Add separate flag for redirecting worker output.

* Fix tests.
2018-03-20 18:14:54 -07:00
Robert Nishihara
794f547d0a Always send actor creation tasks to the global scheduler. (#1757) 2018-03-20 14:55:20 -07:00
Robert Nishihara
4658d0a180 Print error when actor takes too long to start, and refactor error me… (#1747)
* Print error when actor takes too long to start, and refactor error message pushing.

* Print warning every ten seconds.

* Fix linting and tests.

* Fix tests.
2018-03-19 20:24:35 -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
6114b6d20e
Implement the client table for the new GCS (#1674)
* Add subscription callback to CallbackData

* Implement ClientTable

* Hook up ClientTable to AsyncGCSClient

* Add client_info to GCSClient Connect interface

* client table callbacks

* Unit test for client table

* Doc

* Fix idempotency check

* Fix mac build

* Fix memory issues in gcs client test

* Fix disconnection bug

* lint
2018-03-11 19:17:18 -07:00
Philipp Moritz
a683cf2c70 Gcs Asio integration (#1633) 2018-03-04 14:51:04 -08:00
Zhenyu Guo
f1e5789c26 restructure how to organize 3rd party libs (#1630)
* restructure how to organize 3rd party libs

* Minor whitespace changes.

* Fix compilation on Linux.

* Pass around Python executable so that the correct version of Python is used.
2018-03-01 14:29:56 -08:00
Robert Nishihara
0fcceef772 Update logging and check macros. (#1627)
* Update logging and check macros.

* Fix linting.

* Fix RAY_DCHECK and unused variable.

* Fix linting
2018-02-28 15:13:00 -08:00
Alexey Tumanov
844a6afcdd Implement simple random spillback policy. (#1493)
* spillback policy implementation: global + local scheduler

* modernize global scheduler policy state; factor out random number engine and generator

* Minimal version.

* Fix test.

* Make load balancing test less strenuous.
2018-02-13 00:09:35 -08:00
Robert Nishihara
89db7841d2 Update arrow version. (#1512) 2018-02-07 23:05:16 -08: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
Robert Nishihara
3195c6aa63 Fix local scheduler crash when driver creates actor and exits. (#1474)
* Make check failures in redis.cc more informative.

* Fix bug by calling task_table_add_task.

* Add test.
2018-01-26 14:29:53 -08:00
Stephanie Wang
668737f383 Replace actor dummy objects with mock calls to the local scheduler (#1467)
* Replace putting the dummy object with a call to the local scheduler

* Mark dummy objects as locally available
2018-01-26 14:18:45 -08:00
Robert Nishihara
5acc98e629 Update arrow with better dataframe serialization and get rid of custo… (#1413)
* Update arrow with better dataframe serialization and get rid of custom dataframe serializers.

* Update plasma client API.

* Fix potential bug.

* Bug fix.

* Update arrow to use deduplicated file descriptors and mutable buffers.

* Fix tests.

* Update commit.

* Update commit.

* Update commit.

* Update commit.

* Update commit

* Update commit back to arrow codebase.'
2018-01-24 10:03:29 -08:00
Alexey Tumanov
f1303291b4 Ray scheduler spillback plumbing + mechanism (#1362)
* spillback mechanism and plumbing : adding spillback counter + timestamp

* linting fix

* documentation

* Fix argument name.
2018-01-23 20:18:12 -08:00
Melih Elibol
4b1c8be4fe Fix setting log-level to debug. (#1432) 2018-01-21 21:51:05 -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
Philipp Moritz
3d224c4edf Second Part of Internal API Refactor (#1326) 2017-12-26 16:22:04 -08:00
Robert Nishihara
76b6b4a2d3 When killing worker, release resources before dispatching tasks. (#1327) 2017-12-15 18:12:03 -08:00
Stephanie Wang
12fdb3f53a Convert actor dummy objects to task execution edges. (#1281)
* Define execution dependencies flatbuffer and add to Redis commands

* Convert TaskSpec to TaskExecutionSpec

* Add execution dependencies to Python bindings

* Submitting actor tasks uses execution dependency API instead of dummy argument

* Fix dependency getters and some cleanup for fetching missing dependencies

* C++ convention

* Make TaskExecutionSpec a C++ class

* Convert local scheduler to use TaskExecutionSpec class

* Convert some pointers to references

* Finish conversion to TaskExecutionSpec class

* fix

* Fix

* Fix memory errors?

* Cast flatbuffers GetSize to size_t

* Fixes

* add more retries in global scheduler unit test

* fix linting and cast fbb.GetSize to size_t

* Style and doc

* Fix linting and simplify from_flatbuf.
2017-12-14 20:47:54 -08:00
Robert Nishihara
f75b51d178 Register Common.error with local scheduler extension module. (#1316)
* Register Common.error with local scheduler extension module.

* Add test.
2017-12-13 11:55:54 -08:00
Stephanie Wang
044548bcff Mark the killed as done outside of loop (#1284) 2017-12-02 14:42:16 -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
Eric Liang
9233e496cc Raise exception when getting the task results of workers that died (#1224)
* wip

* with test

* add timeout

* also add test for f

* remove on cleanup

* update

* wip

* fix tests

* mark actor removed in redis

* clang-format

* fix bug when no-inprogress tasks

* try to set task status done

* Add comment.
2017-11-20 15:18:39 -08:00
Peter Schafhalter
d986294c2b Replace UT strings in local scheduler (#1213)
* Convert to string using std::string

* Fix linting issue

* Fix linting

* Construct db_connect_args using vector

* Use vector size() instead of num_args

* Hopefully fix linting now
2017-11-17 16:14:46 -08:00
Stephanie Wang
c70430f322 Fix bugs in plasma manager transfer (#1188)
* Plasma client test for plasma abort

* Use ray-project/arrow:abort-objects branch

* Set plasma manager connection cursor to -1 when not in use

* Handle transfer errors between plasma managers, abort unsealed objects

* Add TODO for local scheduler exiting on plasma manager death

* Revert "Plasma client test for plasma abort"

This reverts commit e00fbd58dc4a632f58383549b19fb9057b305a14.

* Upgrade arrow to version with PlasmaClient::Abort

* Fix plasma manager test

* Fix plasma test

* Temporarily use arrow fork for testing

* fix and set arrow commit

* Fix plasma test

* Fix plasma manager test and make write_object_chunk consistent with read_object_chunk

* style

* upgrade arrow
2017-11-15 22:32:38 -08:00
Peter Schafhalter
9a6a056609 Convert UT datastructures in tests (#1203)
* bind_ipc_sock_retry returns std::string

* snprintf -> std::snprintf

* Fix formatting

* Use stringstream instead of snprintf

* Fix typo
2017-11-11 16:55:05 -08:00
Philipp Moritz
e798a652bc Change TaskSpec to allow multiple object IDs per argument. (#1204)
* Implement object ID bags

* linting

* fix tests

* fix linting

* fix comments
2017-11-10 16:33:34 -08:00
Stephanie Wang
07f0532b9b Local scheduler filters out dead clients during reconstruction (#1182)
* Object table lookup returns vector of DBClientID instead of address strings

* Add node IP address to DBClient notification

* DB client cache stores entire DB client, convert addresses to std::string

* get cached db client returns the client

* Expose a call to initialize the redis cache

* Local scheduler filters out dead clients during reconstruction

* Remove node ip address from dbclient, use aux_address for plasma managers

* Get entire db client entry when not found in cache

* Fix common tests

* Fix address in tests

* Push error to driver if driver task did the put

* Address Robert's comments and cleanup

* Remove unused Redis command

* Fix db test
2017-11-10 11:29:24 -08:00
Robert Nishihara
1c6b30b5e2 Move all config constants into single file. (#1192)
* Initial pass at factoring out C++ configuration into a single file.

* Expose config through Python.

* Forward declarations.

* Fixes with Python extensions

* Remove old code.

* Consistent naming for constants.

* Fixes

* Fix linting.

* More linting.

* Whitespace

* rename config -> _config.

* Move config inside a class.

* update naming convention

* Fix linting.

* More linting

* More linting.

* Add in some more constants.

* Fix linting
2017-11-08 11:10:38 -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
Robert Nishihara
1cdc2fb011 Clean up event loop and callbacks when processes exit. (#1125)
* Clean up event loop and callbacks when processes exit.

* Fix bug.
2017-10-19 17:07:03 -07:00
Robert Nishihara
f3e3c7ec71 Add is_actor_checkpoint_method to TaskSpec. (#1117)
* Add is_actor_checkpoint_method to TaskSpec.

* Fix linting.

* Fix rebase error.

* Fix errors from rebase.
2017-10-15 16:52:10 -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
Robert Nishihara
486cb64e3f Compile with -Werror and -Wall (#1116)
* Compile global scheduler with -Werror -Wall.

* Compile plasma manager with -Werror -Wall.

* Compile local scheduler with -Werror -Wall.

* Compile common code with -Werror -Wall.

* Signed/unsigned comparisons.

* More signed/unsigned fixes.

* More signed/unsigned fixes and added extern keyword.

* Fix linting.

* Don't check strict-aliasing because Python.h doesn't pass.
2017-10-12 21:00:23 -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
b585001881 When a task is passed to the global scheduler, if it is not received,… (#1106)
* When a task is passed to the global scheduler, if it is not received, then try again.
* Call give_task_to_global_scheduler directly (same with local).
2017-10-12 00:04:38 -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
Robert Nishihara
1488975d1b Add timing statement to loop that calls redis_get_cached_db_client be… (#1045)
* Add timing statement to loop that calls redis_get_cached_db_client because it has been slow in the past.

* Fix linting.

* Refactoring to make manager vectors into std::vector.

* Fix linting.

* Fixes.
2017-10-02 10:46:21 -07:00
Robert Nishihara
a31d138f21 Don't log when a worker can't be started. (#1056) 2017-10-02 10:32:46 -07:00
Robert Nishihara
ce278aa06a Fix valgrind tests. (#1037)
* Comment out local scheduler valgrind test.

* Fix free/delete error.

* More free -> delete errors

* One more free -> delete and also clean up callback state in plasma manager.

* Add set -x to run_valgrind scripts.

* Fix valgrind error in CreateLocalSchedulerInfoMessage.
2017-09-30 00:11:09 -07:00
Eric Liang
ba153adc4c Downgrade severity of most common messages (#1039)
* downgrade severity of most common messages

* update
2017-09-30 00:01:49 -07:00
Stephanie Wang
74ac80631b Local scheduler sends a null heartbeat to global scheduler (#962)
* Local scheduler sends a null heartbeat to global scheduler to notify death

* Add whitespace.

* Speed up component failures test

* Free local scheduler state upon plasma manager disconnection
2017-09-12 10:45:21 -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
Philipp Moritz
7030ef366f Rebase Ray on latest arrow (remove numbuf from Ray). (#910)
* remove some stuff

* put get roundtrip working

* fixes

* more fixes

* cleanup

* fix tests

* latest arrow

* fixes

* fix tests

* fix linting

* rebase

* fixes

* fix bug

* bring back libgcc error

* fix linting

* use official arrow repo

* fixes
2017-09-04 22:58:49 -07:00
Stephanie Wang
7496c98010 Fault tolerance race (#894)
* Remove race between local scheduler disconnecting and global scheduler
assigning a task

* Fix number of workers started in component failures test

* Fix race between global scheduler retrying a task assignment and monitor
cleaning up task table. The global scheduler should only retry the task
assignment if the local scheduler is still alive.

* Clean up task_table_update callback if failure

* Look up current local scheduler mapping when retrying actor task submission

* Log warning if no subscribers received a task table update

* Clean up database handle memory in local scheduler
2017-08-30 22:20:50 -07:00
Alexey Tumanov
fc885bd918 Adding basic support for a user-interpretable resource label (#761)
* adding support for the user-interpretable label(UIR)

* more plumbing for num_uirs further upstream; set to infty when specified on cmd line

* pass default num_uirs for actors; update GlobalStateAPI

* support num_uirs in ray.init()

* local scheduler resource accounting: support num_uirs; prep for vectorized resource accounting

* global scheduler test updated

* Fix bug introduced by rebase.

* Rename UIR -> CustomResource and add test.

* Small changes and use constexpr instead of macros.

* Linting and some renaming.

* Reorder some code.

* Remove cpus_in_use and fix bug.

* Add another test and make a small change.

* Rephrase documentation about feature stability.
2017-08-08 02:53:59 -07:00