Commit graph

194 commits

Author SHA1 Message Date
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
a8032b9ca1 Convert connections from UT_array to std::vector (#1190) 2017-11-07 20:59:41 -08:00
Peter Schafhalter
7215f7d228 Remove UT String from logging (#1184)
* Removed unnecessary utarray include

* Removed ut_string from logging

* Fix formatting
2017-11-05 14:05:20 -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
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
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
Robert Nishihara
d6062ef8f6 Compile with -rdynamic for better debugging symbols. (#1123)
* Compile with -rdynamic.

* Only use -rdynamic on Linux.

* Add comment.
2017-10-13 21:39:11 -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
1e0ab3d386 Switch to monotonic clock (#1100) 2017-10-10 22:35:21 -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
Zongheng Yang
427dee511b Fill out specs of the task table in ray_redis_module.cc. (#1024)
* Fill out specs of the task table in ray_redis_module.cc.

* local scheduler field in task table

* linting
2017-09-27 23:45:58 -07:00
Zongheng Yang
5a50e80b63 Make Monitor remove dead Redis entries from exiting drivers. (#994)
* WIP: removing OL, OI, TT on client exit; no saving yet.

* ray_redis_module.cc: update header comment.

* Cleanup: just the removal.

* Reformat via yapf: use pep8 style instead of google.

* Checkpoint addressing comments (partially)

* Add 'b' marker before strings (py3 compat)

* Add MonitorTest.

* Use `isort` to sort imports.

* Remove some loggings

* Fix flake8 noqa marker runtest.py

* Try to separate tests out to monitor_test.py

* Rework cleanup algorithm: correct logic

* Extend tests to cover multi-shard cases

* Add some small comments and formatting changes.
2017-09-26 00:11:38 -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
ae0212b399 Fix failing task table test (#924) 2017-09-03 22:41:38 -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
be4beb19c1 Changes to build to fix creation of wheels. (#840)
* Pass DPYTHON_EXECUTABLE into cmake for arrow and for ray.

* Add cython to setup.py install_requires.

* Revert custom code for finding python in cmake.

* Correctly find arrow on CentOS.

* In cmake, don't find PythonLibs, just find PYTHON_INCLUDE_DIRS.

* Fix typo.

* Do not use boost shared libraries when building arrow.

* Add six to the setup.py install_requires because it is needed by pyarrow.

* Don't link numbuf against boost_system and boost_filesystem.

* Compile boost when we are on Linux.

* Make numbuf find the correct boost libraries.

* Only use find_package Boost on Linux, suppress output when building boost.

* Changes to wheel building scripts, install cython in mac script.

* Compile flatbuffers ourselves on Linux and pass it in when compiling Arrow.

* Clean up build_flatbuffers.sh and build_boost.sh scripts a little.

* Install cython when building linux wheel.
2017-08-21 17:49:35 -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
Philipp Moritz
054ae4180e Fix installation instruction for ubuntu 14.04 (#805)
* fix installation instruction for ubuntu 14.04

* upgrade cmake requirements

* fix
2017-08-02 18:14:14 -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
8c8258de20 Move worker methods into Worker class and expose more TaskSpec fields to Python. (#796)
* Move worker methods inside worker class. Move some helper methods from actor.py into utils.py and state.py.

* Add more methods exposing task spec fields to Python.

* Fix linting.

* Fix error.

* Remove unused code in default worker.
2017-08-01 17:16:57 -07:00
Philipp Moritz
c3b39b4d86 Pull Plasma from Apache Arrow and remove Plasma store from Ray. (#692)
* Rebase Ray on top of Plasma in Apache Arrow

* add thirdparty building scripts

* use rebased arrow

* fix

* fix build

* fix python visibility

* comment out C tests for now

* fix multithreading

* fix

* reduce logging

* fix plasma manager multithreading

* make sure old and new object IDs can coexist peacefully

* more rebasing

* update

* fixes

* fix

* install pyarrow

* install cython

* fix

* install newer cmake

* fix

* rebase on top of latest arrow

* getting runtest.py run locally (needed to comment out a test for that to work)

* work on plasma tests

* more fixes

* fix local scheduler tests

* fix global scheduler test

* more fixes

* fix python 3 bytes vs string

* fix manager tests valgrind

* fix documentation building

* fix linting

* fix c++ linting

* fix linting

* add tests back in

* Install without sudo.

* Set PKG_CONFIG_PATH in build.sh so that Ray can find plasma.

* Install pkg-config

* Link -lpthread, note that find_package(Threads) doesn't seem to work reliably.

* Comment in testGPUIDs in runtest.py.

* Set PKG_CONFIG_PATH when building pyarrow.

* Pull apache/arrow and not pcmoritz/arrow.

* Fix installation in docker image.

* adapt to changes of the plasma api

* Fix installation of pyarrow module.

* Fix linting.

* Use correct python executable to build pyarrow.
2017-07-31 21:04:15 -07:00
Robert Nishihara
8ad9ced99b Fix task ID hash computation. (#774) 2017-07-26 10:08:38 -07:00
alanamarzoev
8464d77c76 Change event logs to store one Redis ZSET per worker. (#705)
* Changing to zset

* Fixed bug.

* Fixed another bug.

* Modified task_profiles.

* Removed extra file.

* Modified task_profiles test.

* WIP

* WIP

* Undid changes

* Updated

* WIP

* Made changes according to comments.

* Removed unneeded print.

* Removed ujson usage.

* failing test

* tests passing

* Fixed linting errors and modified style.

* Fixed bug.

* Fixed linting

* Fixed according to comments.

* Redis crashing?

* Fixed linting

* Fixed linting
2017-07-09 01:42:29 +02: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
alanamarzoev
4d5ac9dad5 Include object size and hash in the table returned by the object_table function in the GlobalStateAPI. (#665)
* added log_table function and a test

* fixed log_files and added task_profiles

* fixed formatting

* fixed linting errors

* fixes

* removed file

* more fixes

* hopefully fixed

* Small changes.

* Fix linting.

* Fix bug in log monitor.

* Small changes.

* Fix bug in travis.

* Including data_size and hash in the ResultTableReply.

* Included data_size and hash info in object_table.

* Fixed bugs in ray_redis_module.cc.

* Removing commented out code.

* Fixes

* Freed hash and data_size strings after using, and checked if they're null along with task_id and is_put.

* Changed it so that data_size is set correctly.

* Removed iostream import.

* Included a check to ensure that the Redis string to long long conversion was successful.

* Included separate data_size and hash null checks.

* Fixed bug.

* Made linting changes.

* Another linting error.

* Slight simplication.
2017-06-16 23:17:11 -07:00
Robert Nishihara
96962cdee0 Log fatal error if plasma manager or local scheduler heartbeats take too long. (#676)
* Log fatal error if plasma manager or local scheduler take too long to send heartbeat.

* Fix linting.

* Use int64_t for milliseconds since unix epoch.
2017-06-16 19:11:01 +00:00
Robert Nishihara
1916475e14 Increase socket listen backlog from 5 to 128. (#661) 2017-06-11 06:34:16 +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
Richard Shin
16050eca8d Don't link Python extensions to libpython*.so (#598) 2017-05-25 19:01:12 -07:00
Philipp Moritz
3885d1b286 make builds with CMake incremental (#592) 2017-05-24 21:52:33 -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
Philipp Moritz
08e988aee5 Modernize plasma store (C to C++ changes). (#546) 2017-05-15 01:19:44 -07:00
Philipp Moritz
1dddd5336a Fix actor bug arising from overwriting task specifications in the local scheduler (#513)
* copy task specifications put into the actor task cache so it won't get overwritten when the scheduler receives the next task

* cleanup

* cleanup and fix

* linting

* fix jenkins test

* fix linting
2017-05-06 17:39:35 -07:00
Stephanie Wang
e50a23b820 Fix bug with reused file descriptors (#471)
* Fix bug with reused file descriptors

* Remove client connection if write_object_chunk fails

* Handle ECONNRESET on unsuccessful write

* lint

* Back to lowercase

* fix compilation

* fix linting
2017-05-02 19:45:27 -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
Alexey Tumanov
a67a107e0e Fix int-type compilation problem on redhat. (#472) 2017-04-19 02:43:33 -07:00
Richard Shin
cf68cf743c Change UniqueID hash function to look at the lowest instead of highest bytes. (#469) 2017-04-18 15:31:49 -07:00
Robert Nishihara
05fd4c2c37 Changes to local scheduler client protocol. (#435)
* Make local scheduler clients receive reply upon registration.

* Fix tests and linting.
2017-04-07 23:03:37 -07:00
Alexey Tumanov
6f9225490b Plasma manager performance: speed up wait with a wait request object map (#427)
* plasma manager perf: speedup wait with a wait request object map

* removing duplicate == operator in plasma store

* fix serialization test

* code cleanup

* minor cleanup

* factoring out uniqueid hash and equality operators into common

* plasma manager: c++ify the WaitRequest struct

* plasma manager: get rid of the initial object request malloc

* cleanup

* linting

* cleanups and fix compiler warnings

* compiler warnings and linting
2017-04-07 12:32:12 -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
Robert Nishihara
f1b48f2fd4 Avoid publishing in the task table unnecessarily. (#416) 2017-03-30 13:41:32 -07:00
Stephanie Wang
036b873bf2 Implement local scheduler task queues using C++ data structures (#392)
* Switch to using C++ lists for task queues

* Init and free methods for TaskQueueEntry

* Switch from utarray to c++ vector for TaskQueueEntry

* Get rid of some pointers

* Back to O(1) deletion from waiting_task_queue

* Fix comments

* Cut code

* Non const iterators

* Fix Alexey's comments
2017-03-30 00:40:01 -07:00
Alexey Tumanov
78e1167a42 Parallelize make in build.sh. (#371)
* parallelize build.sh make

* Encode in cmake the dependency of ray_redis_module on autogenerated flatbuffer files.
2017-03-27 20:55:50 -07:00