Commit graph

83 commits

Author SHA1 Message Date
yuyiming
9ff3d57429 do not fetch from dead Plasma Manager (#2116) 2018-05-23 16:13:09 -07:00
eric-jj
34bc6ce6ea remove UniqueIDHasher (#1957)
* remove UniqueIDHasher

* Format the change

* remove unused line

* Fix format

* fix lint error

* fix linting whitespace
2018-04-30 06:31:23 -07:00
Robert Nishihara
cffda73da1 Allow task_table_update to fail when tasks are finished. (#1927)
* Allow task_table_update to fail when tasks are finished.

* Add comment.
2018-04-20 11:34:29 -07:00
Melih Elibol
cff37765b1 Addresses missed comments from multichunk object transfer PR. (#1908)
* Move object manager parameters to ray config,
object manager config bug fix.
addresses other comments from #1827.

* linting and uint?

* typos

* remove uint.
2018-04-15 21:35:51 -07:00
Robert Nishihara
6ca2c2a609 Allow numpy arrays to be passed by value into tasks (and inlined in the task spec). (#1816)
* Allow numpy arrays and larger objects to be passed by value in task specifications.

* Fix bug.

* Fix bug. Inline all bug numpy object arrays.

* Increase size limit for inlining args in task spec.

* Give numpy init different signatures in Python 2 and Python 3.

* Simplify code.

* Fix test.

* Use import_array1 instead of import_array.
2018-04-15 20:36:01 -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
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
Philipp Moritz
1ab2e63dbd Tune transfer buffer size (#1363)
Increase buffsize from `4096` to `80*1024`.
2018-02-09 14:56:36 -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
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
Robert Nishihara
088f01496c Remove unused object info table code. (#1388) 2018-01-05 11:00:06 -08:00
Philipp Moritz
3d224c4edf Second Part of Internal API Refactor (#1326) 2017-12-26 16:22:04 -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
Stephanie Wang
bac39a134e
Define a wrapper class for callback_data.data (#1301) 2017-12-08 11:48:21 -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
e0a340ee7e Allow actors to pin at most 1000 dummy objects at a time. (#1241)
* Allow actors to pin at most 1000 dummy objects at a time.

* Fix linting.
2017-11-22 13:38:01 -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
e0360eb429 Remove UT libraries and clean up remaining UT datastructures (#1230)
* Remove UT string include from redis

* Remove UT string include from DB tests

* Modify TaskSpec_print to remove UT string

* Remove UT libraries
2017-11-19 15:01:33 -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
d3c082d325 More checking in redis.cc. (#1057) 2017-11-08 23:25:19 -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
Peter Schafhalter
ad4cbd4016 Updated outstanding_callbacks to unordered_map (#1108)
* Updated outstanding_callbacks to unordered_map

* Fix bug in destroy_outstanding_callbacks and comments
2017-10-20 10:06:22 -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
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
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
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
Peter Schafhalter
2c19ae97a3 Implemented db_client_cache as unordered_map (#921)
* Implemented db_client_cache as unordered_map

* Fix for memory leak

* Fixed linting
2017-09-03 17:26:05 -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
Robert Nishihara
e6de744ef4 Fix potential bug in redis.cc. (#851) 2017-08-23 20:38:25 -07:00
Robert Nishihara
cb84972f6b Recreate actors when local schedulers die. (#804)
* Reconstruct actor state when local schedulers fail.

* Simplify construction of arguments to pass into default_worker.py from local scheduler.

* Remove deprecated ray.actor.

* Simplify actor reconstruction method.

* Fix linting.

* Small fixes.
2017-08-02 18:02:52 -07:00
Robert Nishihara
9e4a3e4972 Replace some UT data structures in local scheduler with C++ STL. (#680)
* Replace a local scheduler ut_array with a std::vector.

* Replace vector of sizes in local scheduler with std::pair.

* Remove utarray include.

* Replace utarray with std::vector for reading local scheduler input messages.

* Remove more UT data structures.

* Remove UT includes.

* Fix linting.

* Include stdlib.h to find size_t.

* Remove includes of stdbool.h.

* Replace std::pair with TaskQueueEntry.

* Fix redis tests.

* Reinstate tests.
2017-06-19 21:58:42 +00:00
Eric Liang
d4d2c03ac5 Remove timeout for Redis commands. (#649)
* update

* Remove interaction between callback data identifier and event loop.

* Remove tests that no longer apply.
2017-06-09 15:55:36 -07:00
Stephanie Wang
ee08c8274b Shard Redis. (#539)
* Implement sharding in the Ray core

* Single node Python modifications to do sharding

* Do the sharding in redis.cc

* Pipe num_redis_shards through start_ray.py and worker.py.

* Use multiple redis shards in multinode tests.

* first steps for sharding ray.global_state

* Fix problem in multinode docker test.

* fix runtest.py

* fix some tests

* fix redis shard startup

* fix redis sharding

* fix

* fix bug introduced by the map-iterator being consumed

* fix sharding bug

* shard event table

* update number of Redis clients to be 64K

* Fix object table tests by flushing shards in between unit tests

* Fix local scheduler tests

* Documentation

* Register shard locations in the primary shard

* Add plasma unit tests back to build

* lint

* lint and fix build

* Fix

* Address Robert's comments

* Refactor start_ray_processes to start Redis shard

* lint

* Fix global scheduler python tests

* Fix redis module test

* Fix plasma test

* Fix component failure test

* Fix local scheduler test

* Fix runtest.py

* Fix global scheduler test for python3

* Fix task_table_test_and_update bug, from actor task table submission race

* Fix jenkins tests.

* Retry Redis shard connections

* Fix test cases

* Convert database clients to DBClient struct

* Fix race condition when subscribing to db client table

* Remove unused lines, add APITest for sharded Ray

* Fix

* Fix memory leak

* Suppress ReconstructionTests output

* Suppress output for APITestSharded

* Reissue task table add/update commands if initial command does not publish to any subscribers.

* fix

* Fix linting.

* fix tests

* fix linting

* fix python test

* fix linting
2017-05-18 17:40:41 -07:00
Robert Nishihara
0ac125e9b2 Clean up when a driver disconnects. (#462)
* Clean up state when drivers exit.

* Remove unnecessary field in ActorMapEntry struct.

* Have monitor release GPU resources in Redis when driver exits.

* Enable multiple drivers in multi-node tests and test driver cleanup.

* Make redis GPU allocation a redis transaction and small cleanups.

* Fix multi-node test.

* Small cleanups.

* Make global scheduler take node_ip_address so it appears in the right place in the client table.

* Cleanups.

* Fix linting and cleanups in local scheduler.

* Fix removed_driver_test.

* Fix bug related to vector -> list.

* Fix linting.

* Cleanup.

* Fix multi node tests.

* Fix jenkins tests.

* Add another multi node test with many drivers.

* Fix linting.

* Make the actor creation notification a flatbuffer message.

* Revert "Make the actor creation notification a flatbuffer message."

This reverts commit af99099c8084dbf9177fb4e34c0c9b1a12c78f39.

* Add comment explaining flatbuffer problems.
2017-04-24 18:10:21 -07:00
Robert Nishihara
fa363a5a3a Notify driver when a worker dies while executing a task. (#419)
* Notify driver when a worker dies while executing a task.

* Fix linting.

* Don't push error when local scheduler is cleaning up.
2017-04-06 00:02:39 -07:00
Stephanie Wang
93679df724 Stopped nodes can rejoin immediately (#428)
* Ignore deleted clients when reading address info from Redis

* Remove self from db_client table when exiting cleanly

* Fix valgrind test

* Do not call plasma_perform_release when disconnecting
2017-04-05 23:50:38 -07:00
Stephanie Wang
083e7a28ad Push an error to the driver when the workload hangs on ray.put reconstruction (#382)
* Fix worker blocked bug

* tmp

* Push an error to the driver on ray.put for non-driver tasks

* Fix result table tests

* Fix test, logging

* Address comments

* Fix suppression bug

* Fix redis module test

* Edit error message

* Get values in chunks during reconstruction

* Test case for driver ray.put errors

* Error for evicting ray.put objects from the driver

* Fix tests

* Reduce verbosity

* Documentation
2017-03-21 00:16:48 -07:00
Stephanie Wang
12c9618c0c Plasma and worker node failure. (#373)
* Failing test case

* Local scheduler exits cleanly after plasma store dies

* Tolerate one plasma store failure

* Tolerate plasma store failures on all nodes except head node

* Plasma manager heartbeats

* Component failure tests

* Don't run the helper for Python testing

* Fix C test

* Fix hanging plasma transfer test

* Fix python3

* Consolidate ClientConnection code

* Fix valgrind test

* fix c test

* We can restart worker nodes!

* Fix flatbuffers bug

* Address comments

* Only register actual workers with the local scheduler

* Fix bug

* Fix segfaults

* Add test case that tests for driver liveness, fix local scheduler bug

* Clean up after tests

* Allocate retry info on the stack

* Send SIGKILL before waiting

* Relax unit test conditions

* Driver liveness test case and documentation
2017-03-17 17:03:58 -07:00
Philipp Moritz
068429ffd8 Convert local scheduler messages to flatbuffers (#340)
* use flatbuffer messages for local scheduler

* make sure constructor gets called for C++ object ObjectInfoT

* fix typo

* fix Robert's comments

* Small change to actor test.

* fix valgrind error

* linting

* free notification

* fix

* valgrind

* fix valgrind

* fix other bugs

* valgrind fix

* fixes

* more fixes

* Small changes to comments.
2017-03-15 16:27:52 -07:00
Robert Nishihara
53dffe0bf2 Use flatbuffers for some messages from Redis. (#341)
* Compile the Ray redis module with C++.

* Redo parsing of object table notifications with flatbuffers.

* Update redis module python tests.

* Redo parsing of task table notifications with flatbuffers.

* Fix linting.

* Redo parsing of db client notifications with flatbuffers.

* Redo publishing of local scheduler heartbeats with flatbuffers.

* Fix linting.

* Remove usage of fixed-width formatting of scheduling state in channel name.

* Reply with flatbuffer object to task table queries, also simplify redis string to flatbuffer string conversion.

* Fix linting and tests.

* fix

* cleanup

* simplify logic in ReplyWithTask
2017-03-10 18:35:25 -08:00
Stephanie Wang
da06b4db82 Warn the user when a nondeterministic task is detected. (#339)
* WARN instead of FATAL for object hash mismatches, push error to driver

* Document the callback signature for object_table_add/remove

* Error table

* Wait for all errors in python test

* Fix doc

* Fix state test
2017-03-07 00:32:15 -08:00
Philipp Moritz
0b8d279ef2 Convert task_spec to flatbuffers (#255)
* convert Ray to C++

* convert task_spec to flatbuffers

* fix

* it compiles

* latest

* tests are passing

* task2 -> task

* fix

* fix

* fix

* fix

* fix

* linting

* fix valgrind

* upgrade flatbuffers

* use debug mode for valgrind

* fix naming and comments

* downgrade flatbuffers

* fix linting

* reintroduce TaskSpec_free

* rename TaskSpec -> TaskInfo

* refactoring

* linting
2017-03-05 02:05:02 -08:00
Stephanie Wang
41b8675d04 Availability after local scheduler failure (#329)
* Clean up plasma subscribers on EPIPE

First pass at a monitoring script - monitor can detect local scheduler death

Clean up task table upon local scheduler death in monitoring script

Don't schedule to dead local schedulers in global scheduler

Have global scheduler update the db clients table, monitor script cleans up state

Documentation

Monitor script should scan tables before beginning to read from subscription channel

Fix for python3

Redirect monitor output to redis logs, fix hanging in multinode tests

* Publish auxiliary addresses as part of db_client deletion notifications

* Fix test case?

* Small changes.

* Use SCAN instead of KEYS

* Address comments

* Address more comments

* Free redis module strings
2017-03-02 19:51:20 -08:00
Philipp Moritz
793a102846 Make Ray code C++ compatible (#321)
* convert Ray to C++
* const correctness
2017-03-01 01:17:24 -08:00
Philipp Moritz
a30eed452e Change type naming convention. (#315)
* Rename object_id -> ObjectID.

* Rename ray_logger -> RayLogger.

* rename task_id -> TaskID, actor_id -> ActorID, function_id -> FunctionID

* Rename plasma_store_info -> PlasmaStoreInfo.

* Rename plasma_store_state -> PlasmaStoreState.

* Rename plasma_object -> PlasmaObject.

* Rename object_request -> ObjectRequests.

* Rename eviction_state -> EvictionState.

* Bug fix.

* rename db_handle -> DBHandle

* Rename local_scheduler_state -> LocalSchedulerState.

* rename db_client_id -> DBClientID

* rename task -> Task

* make redis.c C++ compatible

* Rename scheduling_algorithm_state -> SchedulingAlgorithmState.

* Rename plasma_connection -> PlasmaConnection.

* Rename client_connection -> ClientConnection.

* Fixes from rebase.

* Rename local_scheduler_client -> LocalSchedulerClient.

* Rename object_buffer -> ObjectBuffer.

* Rename client -> Client.

* Rename notification_queue -> NotificationQueue.

* Rename object_get_requests -> ObjectGetRequests.

* Rename get_request -> GetRequest.

* Rename object_info -> ObjectInfo.

* Rename scheduler_object_info -> SchedulerObjectInfo.

* Rename local_scheduler -> LocalScheduler and some fixes.

* Rename local_scheduler_info -> LocalSchedulerInfo.

* Rename global_scheduler_state -> GlobalSchedulerState.

* Rename global_scheduler_policy_state -> GlobalSchedulerPolicyState.

* Rename object_size_entry -> ObjectSizeEntry.

* Rename aux_address_entry -> AuxAddressEntry.

* Rename various ID helper methods.

* Rename Task helper methods.

* Rename db_client_cache_entry -> DBClientCacheEntry.

* Rename local_actor_info -> LocalActorInfo.

* Rename actor_info -> ActorInfo.

* Rename retry_info -> RetryInfo.

* Rename actor_notification_table_subscribe_data -> ActorNotificationTableSubscribeData.

* Rename local_scheduler_table_send_info_data -> LocalSchedulerTableSendInfoData.

* Rename table_callback_data -> TableCallbackData.

* Rename object_info_subscribe_data -> ObjectInfoSubscribeData.

* Rename local_scheduler_table_subscribe_data -> LocalSchedulerTableSubscribeData.

* Rename more redis call data structures.

* Rename photon_conn PhotonConnection.

* Rename photon_mock -> PhotonMock.

* Fix formatting errors.
2017-02-26 00:32:43 -08:00
Stephanie Wang
be1618f041 Availability after worker failure (#316)
* Availability after a killed worker

* Workers exit cleanly

* Memory cleanup in photon C tests

* Worker failure in multinode

* Consolidate worker cleanup handlers

* Update the result table before handling a task submission

* KILL_WORKER_TIMEOUT -> KILL_WORKER_TIMEOUT_MILLISECONDS

* Log a warning instead of crashing if no result table entry found
2017-02-25 20:19:36 -08:00