Skip to content

Commit

Permalink
[core worker] Python core worker task execution (ray-project#5783)
Browse files Browse the repository at this point in the history
Executes tasks via the event loop in the C++ core worker. Also properly handles signals (including KeyboardInterrupt), so ctrl-C in a python interactive shell works now (if connecting to an existing cluster).
  • Loading branch information
edoakes authored Oct 23, 2019
1 parent 95241f6 commit 02931e0
Show file tree
Hide file tree
Showing 38 changed files with 830 additions and 678 deletions.
421 changes: 354 additions & 67 deletions python/ray/_raylet.pyx

Large diffs are not rendered by default.

4 changes: 2 additions & 2 deletions python/ray/experimental/async_plasma.py
Original file line number Diff line number Diff line change
Expand Up @@ -199,8 +199,8 @@ def _unregister_callback(self, fut):
del self._waiting_dict[fut.object_id]

def _complete_future(self, fut):
obj = self._worker.retrieve_and_deserialize(
[ray.ObjectID(fut.object_id.binary())], 0)[0]
obj = self._worker.get_objects([ray.ObjectID(
fut.object_id.binary())])[0]
fut.set_result(obj)

def as_future(self, object_id, check_ready=True):
Expand Down
7 changes: 3 additions & 4 deletions python/ray/experimental/signal.py
Original file line number Diff line number Diff line change
Expand Up @@ -69,11 +69,10 @@ def send(signal):
Args:
signal: Signal to be sent.
"""
if hasattr(ray.worker.global_worker, "actor_creation_task_id"):
source_key = ray.worker.global_worker.actor_id.hex()
else:
# No actors; this function must have been called from a task
if ray.worker.global_worker.actor_id.is_nil():
source_key = ray.worker.global_worker.current_task_id.hex()
else:
source_key = ray.worker.global_worker.actor_id.hex()

encoded_signal = ray.utils.binary_to_hex(cloudpickle.dumps(signal))
ray.worker.global_worker.redis_client.execute_command(
Expand Down
2 changes: 1 addition & 1 deletion python/ray/function_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -763,7 +763,7 @@ def _make_actor_method_executor(self, method_name, method, actor_imported):
worker's internal state to record the executed method.
"""

def actor_method_executor(dummy_return_id, actor, *args, **kwargs):
def actor_method_executor(actor, *args, **kwargs):
# Update the actor's task counter to reflect the task we're about
# to execute.
self._worker.actor_task_counter += 1
Expand Down
32 changes: 22 additions & 10 deletions python/ray/includes/common.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -47,31 +47,34 @@ cdef extern from "ray/common/status.h" namespace "ray" nogil:
CRayStatus OK()

@staticmethod
CRayStatus OutOfMemory()
CRayStatus OutOfMemory(const c_string &msg)

@staticmethod
CRayStatus KeyError()
CRayStatus KeyError(const c_string &msg)

@staticmethod
CRayStatus Invalid()
CRayStatus Invalid(const c_string &msg)

@staticmethod
CRayStatus IOError()
CRayStatus IOError(const c_string &msg)

@staticmethod
CRayStatus TypeError()
CRayStatus TypeError(const c_string &msg)

@staticmethod
CRayStatus UnknownError()
CRayStatus UnknownError(const c_string &msg)

@staticmethod
CRayStatus NotImplemented()
CRayStatus NotImplemented(const c_string &msg)

@staticmethod
CRayStatus RedisError()
CRayStatus ObjectStoreFull(const c_string &msg)

@staticmethod
CRayStatus ObjectStoreFull()
CRayStatus RedisError(const c_string &msg)

@staticmethod
CRayStatus Interrupted(const c_string &msg)

c_bool ok()
c_bool IsOutOfMemory()
Expand All @@ -81,8 +84,9 @@ cdef extern from "ray/common/status.h" namespace "ray" nogil:
c_bool IsTypeError()
c_bool IsUnknownError()
c_bool IsNotImplemented()
c_bool IsRedisError()
c_bool IsObjectStoreFull()
c_bool IsRedisError()
c_bool IsInterrupted()

c_string ToString()
c_string CodeAsString()
Expand All @@ -92,6 +96,7 @@ cdef extern from "ray/common/status.h" namespace "ray" nogil:
# We can later add more of the common status factory methods as needed
cdef CRayStatus RayStatus_OK "Status::OK"()
cdef CRayStatus RayStatus_Invalid "Status::Invalid"()
cdef CRayStatus RayStatus_NotImplemented "Status::NotImplemented"()


cdef extern from "ray/common/status.h" namespace "ray::StatusCode" nogil:
Expand All @@ -117,6 +122,8 @@ cdef extern from "ray/protobuf/common.pb.h" nogil:
pass
cdef cppclass CWorkerType "ray::WorkerType":
pass
cdef cppclass CTaskType "ray::TaskType":
pass


# This is a workaround for C++ enum class since Cython has no corresponding
Expand All @@ -130,6 +137,11 @@ cdef extern from "ray/protobuf/common.pb.h" nogil:
cdef CWorkerType WORKER_TYPE_WORKER "ray::WorkerType::WORKER"
cdef CWorkerType WORKER_TYPE_DRIVER "ray::WorkerType::DRIVER"

cdef extern from "ray/protobuf/common.pb.h" nogil:
cdef CTaskType TASK_TYPE_NORMAL_TASK "ray::TaskType::NORMAL_TASK"
cdef CTaskType TASK_TYPE_ACTOR_CREATION_TASK "ray::TaskType::ACTOR_CREATION_TASK" # noqa: E501
cdef CTaskType TASK_TYPE_ACTOR_TASK "ray::TaskType::ACTOR_TASK"


cdef extern from "ray/common/task/scheduling_resources.h" nogil:
cdef cppclass ResourceSet "ray::ResourceSet":
Expand Down
41 changes: 38 additions & 3 deletions python/ray/includes/libcoreworker.pxd
Original file line number Diff line number Diff line change
@@ -1,7 +1,13 @@
# cython: profile = False
# distutils: language = c++
# cython: embedsignature = True

from libc.stdint cimport int64_t
from libcpp cimport bool as c_bool
from libcpp.memory cimport shared_ptr, unique_ptr
from libcpp.string cimport string as c_string
from libcpp.unordered_map cimport unordered_map
from libcpp.utility cimport pair
from libcpp.vector cimport vector as c_vector

from ray.includes.unique_ids cimport (
Expand All @@ -18,12 +24,30 @@ from ray.includes.common cimport (
CRayStatus,
CTaskArg,
CTaskOptions,
CTaskType,
CWorkerType,
CLanguage,
CGcsClientOptions,
)
from ray.includes.task cimport CTaskSpec
from ray.includes.libraylet cimport CRayletClient

ctypedef unordered_map[c_string, c_vector[pair[int64_t, double]]] \
ResourceMappingType

cdef extern from "ray/core_worker/task_execution.h" namespace "ray" nogil:
cdef cppclass CTaskExecutionInterface "CoreWorkerTaskExecutionInterface":
void Run()
void Stop()

cdef extern from "ray/core_worker/profiling.h" nogil:
cdef cppclass CProfiler "ray::worker::Profiler":
void Start()

cdef cppclass CProfileEvent "ray::worker::ProfileEvent":
CProfileEvent(const shared_ptr[CProfiler] profiler,
const c_string &event_type)
void SetExtraData(const c_string &extra_data)

cdef extern from "ray/core_worker/profiling.h" nogil:
cdef cppclass CProfileEvent "ray::worker::ProfileEvent":
Expand Down Expand Up @@ -54,12 +78,23 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
const c_string &raylet_socket, const CJobID &job_id,
const CGcsClientOptions &gcs_options,
const c_string &log_dir, const c_string &node_ip_address,
void* execution_callback,
CRayStatus (
CTaskType task_type,
const CRayFunction &ray_function,
const CJobID &job_id,
const CActorID &actor_id,
const unordered_map[c_string, double] &resources,
const c_vector[shared_ptr[CRayObject]] &args,
const c_vector[CObjectID] &arg_reference_ids,
const c_vector[CObjectID] &return_ids,
c_vector[shared_ptr[CRayObject]] *returns) nogil,
CRayStatus() nogil,
c_bool use_memory_store_)
void Disconnect()
CWorkerType &GetWorkerType()
CLanguage &GetLanguage()
CObjectInterface &Objects()
CTaskExecutionInterface &Execution()

CRayStatus SubmitTask(
const CRayFunction &function, const c_vector[CTaskArg] &args,
Expand All @@ -72,7 +107,6 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
const c_vector[CTaskArg] &args, const CTaskOptions &options,
c_vector[CObjectID] *return_ids)

# CTaskExecutionInterface &Execution()
unique_ptr[CProfileEvent] CreateProfileEvent(
const c_string &event_type)

Expand All @@ -81,12 +115,13 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
CRayletClient &GetRayletClient()
# TODO(edoakes): remove these once the Python core worker uses the task
# interfaces
CJobID GetCurrentJobId()
void SetCurrentJobId(const CJobID &job_id)
CTaskID GetCurrentTaskId()
void SetCurrentTaskId(const CTaskID &task_id)
void SetActorId(const CActorID &actor_id)
const CActorID &GetActorId()
CTaskID GetCallerId()
const ResourceMappingType &GetResourceIDs() const
CActorID DeserializeAndRegisterActorHandle(const c_string &bytes)
CRayStatus SerializeActorHandle(const CActorID &actor_id, c_string
*bytes)
Expand Down
4 changes: 0 additions & 4 deletions python/ray/includes/libraylet.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@ from libcpp cimport bool as c_bool
from libcpp.memory cimport unique_ptr
from libcpp.string cimport string as c_string
from libcpp.utility cimport pair
from libcpp.unordered_map cimport unordered_map
from libcpp.vector cimport vector as c_vector

from ray.includes.common cimport (
Expand Down Expand Up @@ -38,8 +37,6 @@ cdef extern from "ray/protobuf/gcs.pb.h" nogil:
GCSProfileTableData()


ctypedef unordered_map[c_string, c_vector[pair[int64_t, double]]] \
ResourceMappingType
ctypedef pair[c_vector[CObjectID], c_vector[CObjectID]] WaitResultPair


Expand Down Expand Up @@ -78,4 +75,3 @@ cdef extern from "ray/raylet/raylet_client.h" nogil:
CWorkerID GetWorkerID() const
CJobID GetJobID() const
c_bool IsWorker() const
const ResourceMappingType &GetResourceIDs() const
28 changes: 11 additions & 17 deletions python/ray/includes/task.pxi
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,6 @@ cdef class TaskSpec:
cdef:
unique_ptr[CTaskSpec] task_spec

@staticmethod
cdef make(unique_ptr[CTaskSpec]& task_spec):
cdef TaskSpec self = TaskSpec.__new__(TaskSpec)
self.task_spec.reset(task_spec.release())
return self

@staticmethod
def from_string(const c_string& task_spec_str):
"""Convert a string to a Ray task specification Python object.
Expand Down Expand Up @@ -82,23 +76,23 @@ cdef class TaskSpec:
def arguments(self):
"""Return the arguments for the task."""
cdef:
CTaskSpec*task_spec = self.task_spec.get()
int64_t num_args = task_spec.NumArgs()
int32_t lang = <int32_t>task_spec.GetLanguage()
int64_t num_args = self.task_spec.get().NumArgs()
int32_t lang = <int32_t>self.task_spec.get().GetLanguage()
int count
arg_list = []

if lang == <int32_t>LANGUAGE_PYTHON:
for i in range(num_args):
count = task_spec.ArgIdCount(i)
count = self.task_spec.get().ArgIdCount(i)
if count > 0:
assert count == 1
arg_list.append(
ObjectID(task_spec.ArgId(i, 0).Binary()))
ObjectID(self.task_spec.get().ArgId(i, 0).Binary()))
else:
data = task_spec.ArgData(i)[:task_spec.ArgDataSize(i)]
metadata = task_spec.ArgMetadata(i)[
:task_spec.ArgMetadataSize(i)]
data = self.task_spec.get().ArgData(i)[
:self.task_spec.get().ArgDataSize(i)]
metadata = self.task_spec.get().ArgMetadata(i)[
:self.task_spec.get().ArgMetadataSize(i)]
if metadata == RAW_BUFFER_METADATA:
obj = data
else:
Expand All @@ -111,10 +105,10 @@ cdef class TaskSpec:

def returns(self):
"""Return the object IDs for the return values of the task."""
cdef CTaskSpec *task_spec = self.task_spec.get()
return_id_list = []
for i in range(task_spec.NumReturns()):
return_id_list.append(ObjectID(task_spec.ReturnId(i).Binary()))
for i in range(self.task_spec.get().NumReturns()):
return_id_list.append(
ObjectID(self.task_spec.get().ReturnId(i).Binary()))
return return_id_list

def required_resources(self):
Expand Down
6 changes: 5 additions & 1 deletion python/ray/state.py
Original file line number Diff line number Diff line change
Expand Up @@ -505,14 +505,18 @@ def _profile_table(self, batch_id):
node_ip_address = profile_table_message.node_ip_address

for profile_event_message in profile_table_message.profile_events:
try:
extra_data = json.loads(profile_event_message.extra_data)
except ValueError:
extra_data = {}
profile_event = {
"event_type": profile_event_message.event_type,
"component_id": component_id,
"node_ip_address": node_ip_address,
"component_type": component_type,
"start_time": profile_event_message.start_time,
"end_time": profile_event_message.end_time,
"extra_data": json.loads(profile_event_message.extra_data),
"extra_data": extra_data
}

profile_events.append(profile_event)
Expand Down
2 changes: 1 addition & 1 deletion python/ray/tests/cluster_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ def add_node(self, **node_args):

return node

def remove_node(self, node, allow_graceful=False):
def remove_node(self, node, allow_graceful=True):
"""Kills all processes associated with worker node.
Args:
Expand Down
1 change: 0 additions & 1 deletion python/ray/tests/test_debug_tools.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,4 +47,3 @@ def f():
stdout=subprocess.PIPE,
stderr=subprocess.PIPE)
assert pgrep_command.communicate()[0]
subprocess.call(["pkill", "-f", "gdb.*{}".format(process_name)])
15 changes: 9 additions & 6 deletions python/ray/tests/test_failure.py
Original file line number Diff line number Diff line change
Expand Up @@ -292,7 +292,7 @@ def get_val(self, x):
def test_worker_raising_exception(ray_start_regular):
@ray.remote
def f():
ray.worker.global_worker._get_next_task_from_raylet = None
ray.worker.global_worker.function_actor_manager = None

# Running this task should cause the worker to raise an exception after
# the task has successfully completed.
Expand Down Expand Up @@ -618,12 +618,17 @@ def f():
time.sleep(1000)
return 1

@ray.remote
def h():
time.sleep(1)
ray.get(f.remote())

@ray.remote
def g():
# Sleep so that the f tasks all get submitted to the scheduler after
# the g tasks.
time.sleep(1)
ray.get(f.remote())
ray.get(h.remote())

[g.remote() for _ in range(num_cpus * 4)]
wait_for_errors(ray_constants.WORKER_POOL_LARGE_ERROR, 1)
Expand Down Expand Up @@ -705,8 +710,6 @@ def test_warning_for_dead_node(ray_start_cluster_2_nodes):


def test_raylet_crash_when_get(ray_start_regular):
nonexistent_id = ray.ObjectID.from_random()

def sleep_to_kill_raylet():
# Don't kill raylet before default workers get connected.
time.sleep(2)
Expand All @@ -715,14 +718,14 @@ def sleep_to_kill_raylet():
thread = threading.Thread(target=sleep_to_kill_raylet)
thread.start()
with pytest.raises(ray.exceptions.UnreconstructableError):
ray.get(nonexistent_id)
ray.get(ray.ObjectID.from_random())
thread.join()


def test_connect_with_disconnected_node(shutdown_only):
config = json.dumps({
"num_heartbeats_timeout": 50,
"heartbeat_timeout_milliseconds": 10,
"raylet_heartbeat_timeout_milliseconds": 10,
})
cluster = Cluster()
cluster.add_node(num_cpus=0, _internal_config=config)
Expand Down
2 changes: 1 addition & 1 deletion python/ray/tests/test_multi_node_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ def test_internal_config(ray_start_cluster_head):
worker = cluster.add_node()
cluster.wait_for_nodes()

cluster.remove_node(worker)
cluster.remove_node(worker, allow_graceful=False)
time.sleep(1)
assert ray.cluster_resources()["CPU"] == 2

Expand Down
Loading

0 comments on commit 02931e0

Please sign in to comment.