Skip to content

Commit

Permalink
Use strongly typed IDs in C++. (ray-project#4185)
Browse files Browse the repository at this point in the history
*  Use strongly typed IDs for C++.

* Avoid heap allocation in cython.

* Fix JNI part

* Fix rebase conflict

* Refine

* Remove type check from __init__

* Remove unused constructor declarations.
  • Loading branch information
guoyuhong authored Mar 7, 2019
1 parent b033255 commit b9ea821
Show file tree
Hide file tree
Showing 31 changed files with 484 additions and 334 deletions.
25 changes: 13 additions & 12 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ cdef c_vector[CObjectID] ObjectIDsToVector(object_ids):
ObjectID object_id
c_vector[CObjectID] result
for object_id in object_ids:
result.push_back(object_id.data)
result.push_back(object_id.native())
return result


Expand All @@ -87,11 +87,11 @@ def compute_put_id(TaskID task_id, int64_t put_index):
if put_index < 1 or put_index > kMaxTaskPuts:
raise ValueError("The range of 'put_index' should be [1, %d]"
% kMaxTaskPuts)
return ObjectID(ComputePutId(task_id.data, put_index).binary())
return ObjectID(ComputePutId(task_id.native(), put_index).binary())


def compute_task_id(ObjectID object_id):
return TaskID(ComputeTaskId(object_id.data).binary())
return TaskID(ComputeTaskId(object_id.native()).binary())


cdef c_bool is_simple_value(value, int *num_elements_contained):
Expand Down Expand Up @@ -225,8 +225,8 @@ cdef class RayletClient:
# parameter.
# TODO(suquark): Should we allow unicode chars in "raylet_socket"?
self.client.reset(new CRayletClient(
raylet_socket.encode("ascii"), client_id.data, is_worker,
driver_id.data, LANGUAGE_PYTHON))
raylet_socket.encode("ascii"), client_id.native(), is_worker,
driver_id.native(), LANGUAGE_PYTHON))

def disconnect(self):
check_status(self.client.get().Disconnect())
Expand All @@ -252,22 +252,23 @@ cdef class RayletClient:
TaskID current_task_id=TaskID.nil()):
cdef c_vector[CObjectID] fetch_ids = ObjectIDsToVector(object_ids)
check_status(self.client.get().FetchOrReconstruct(
fetch_ids, fetch_only, current_task_id.data))
fetch_ids, fetch_only, current_task_id.native()))

def notify_unblocked(self, TaskID current_task_id):
check_status(self.client.get().NotifyUnblocked(current_task_id.data))
check_status(self.client.get().NotifyUnblocked(current_task_id.native()))

def wait(self, object_ids, int num_returns, int64_t timeout_milliseconds,
c_bool wait_local, TaskID current_task_id):
cdef:
WaitResultPair result
c_vector[CObjectID] wait_ids
CTaskID c_task_id = current_task_id.native()
wait_ids = ObjectIDsToVector(object_ids)
with nogil:
check_status(self.client.get().Wait(wait_ids, num_returns,
timeout_milliseconds,
wait_local,
current_task_id.data, &result))
c_task_id, &result))
return (VectorToObjectIDs(result.first),
VectorToObjectIDs(result.second))

Expand All @@ -291,9 +292,9 @@ cdef class RayletClient:
postincrement(iterator)
return resources_dict

def push_error(self, DriverID job_id, error_type, error_message,
def push_error(self, DriverID driver_id, error_type, error_message,
double timestamp):
check_status(self.client.get().PushError(job_id.data,
check_status(self.client.get().PushError(driver_id.native(),
error_type.encode("ascii"),
error_message.encode("ascii"),
timestamp))
Expand Down Expand Up @@ -354,7 +355,7 @@ cdef class RayletClient:

def prepare_actor_checkpoint(self, ActorID actor_id):
cdef CActorCheckpointID checkpoint_id
cdef CActorID c_actor_id = actor_id.data
cdef CActorID c_actor_id = actor_id.native()
# PrepareActorCheckpoint will wait for raylet's reply, release
# the GIL so other Python threads can run.
with nogil:
Expand All @@ -365,7 +366,7 @@ cdef class RayletClient:
def notify_actor_resumed_from_checkpoint(self, ActorID actor_id,
ActorCheckpointID checkpoint_id):
check_status(self.client.get().NotifyActorResumedFromCheckpoint(
actor_id.data, checkpoint_id.data))
actor_id.native(), checkpoint_id.native()))

@property
def language(self):
Expand Down
2 changes: 1 addition & 1 deletion python/ray/includes/libraylet.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ cdef extern from "ray/raylet/raylet_client.h" nogil:
int num_returns, int64_t timeout_milliseconds,
c_bool wait_local, const CTaskID &current_task_id,
WaitResultPair *result)
CRayStatus PushError(const CDriverID &job_id, const c_string &type,
CRayStatus PushError(const CDriverID &driver_id, const c_string &type,
const c_string &error_message, double timestamp)
CRayStatus PushProfileEvents(
const GCSProfileTableDataT &profile_events)
Expand Down
18 changes: 8 additions & 10 deletions python/ray/includes/task.pxi
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ cdef class Task:
for arg in arguments:
if isinstance(arg, ObjectID):
references = c_vector[CObjectID]()
references.push_back((<ObjectID>arg).data)
references.push_back((<ObjectID>arg).native())
task_args.push_back(
static_pointer_cast[CTaskArgument,
CTaskArgumentByReference](
Expand All @@ -71,23 +71,21 @@ cdef class Task:

for new_actor_handle in new_actor_handles:
task_new_actor_handles.push_back(
(<ActorHandleID?>new_actor_handle).data)
(<ActorHandleID?>new_actor_handle).native())

self.task_spec.reset(new CTaskSpecification(
CUniqueID(driver_id.data), parent_task_id.data, parent_counter,
actor_creation_id.data, actor_creation_dummy_object_id.data,
max_actor_reconstructions, CUniqueID(actor_id.data),
CUniqueID(actor_handle_id.data), actor_counter,
task_new_actor_handles, task_args, num_returns,
required_resources, required_placement_resources,
LANGUAGE_PYTHON, c_function_descriptor))
driver_id.native(), parent_task_id.native(), parent_counter, actor_creation_id.native(),
actor_creation_dummy_object_id.native(), max_actor_reconstructions, actor_id.native(),
actor_handle_id.native(), actor_counter, task_new_actor_handles, task_args, num_returns,
required_resources, required_placement_resources, LANGUAGE_PYTHON,
c_function_descriptor))

# Set the task's execution dependencies.
self.execution_dependencies.reset(new c_vector[CObjectID]())
if execution_arguments is not None:
for execution_arg in execution_arguments:
self.execution_dependencies.get().push_back(
(<ObjectID?>execution_arg).data)
(<ObjectID?>execution_arg).native())

@staticmethod
cdef make(unique_ptr[CTaskSpecification]& task_spec):
Expand Down
84 changes: 72 additions & 12 deletions python/ray/includes/unique_ids.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,14 @@ from libc.stdint cimport uint8_t
cdef extern from "ray/id.h" namespace "ray" nogil:
cdef cppclass CUniqueID "ray::UniqueID":
CUniqueID()
CUniqueID(const c_string &binary)
CUniqueID(const CUniqueID &from_id)

@staticmethod
CUniqueID from_random()

@staticmethod
CUniqueID from_binary(const c_string & binary)
CUniqueID from_binary(const c_string &binary)

@staticmethod
const CUniqueID nil()
Expand All @@ -26,14 +27,73 @@ cdef extern from "ray/id.h" namespace "ray" nogil:
c_string binary() const
c_string hex() const

ctypedef CUniqueID CActorCheckpointID
ctypedef CUniqueID CActorClassID
ctypedef CUniqueID CActorHandleID
ctypedef CUniqueID CActorID
ctypedef CUniqueID CClientID
ctypedef CUniqueID CConfigID
ctypedef CUniqueID CDriverID
ctypedef CUniqueID CFunctionID
ctypedef CUniqueID CObjectID
ctypedef CUniqueID CTaskID
ctypedef CUniqueID CWorkerID
cdef cppclass CActorCheckpointID "ray::ActorCheckpointID"(CUniqueID):

@staticmethod
CActorCheckpointID from_binary(const c_string &binary)


cdef cppclass CActorClassID "ray::ActorClassID"(CUniqueID):

@staticmethod
CActorClassID from_binary(const c_string &binary)


cdef cppclass CActorID "ray::ActorID"(CUniqueID):

@staticmethod
CActorID from_binary(const c_string &binary)


cdef cppclass CActorHandleID "ray::ActorHandleID"(CUniqueID):

@staticmethod
CActorHandleID from_binary(const c_string &binary)


cdef cppclass CClientID "ray::ClientID"(CUniqueID):

@staticmethod
CClientID from_binary(const c_string &binary)


cdef cppclass CConfigID "ray::ConfigID"(CUniqueID):

@staticmethod
CConfigID from_binary(const c_string &binary)


cdef cppclass CFunctionID "ray::FunctionID"(CUniqueID):

@staticmethod
CFunctionID from_binary(const c_string &binary)


cdef cppclass CDriverID "ray::DriverID"(CUniqueID):

@staticmethod
CDriverID from_binary(const c_string &binary)


cdef cppclass CJobID "ray::JobID"(CUniqueID):

@staticmethod
CJobID from_binary(const c_string &binary)


cdef cppclass CTaskID "ray::TaskID"(CUniqueID):

@staticmethod
CTaskID from_binary(const c_string &binary)


cdef cppclass CObjectID" ray::ObjectID"(CUniqueID):

@staticmethod
CObjectID from_binary(const c_string &binary)


cdef cppclass CWorkerID "ray::WorkerID"(CUniqueID):

@staticmethod
CWorkerID from_binary(const c_string &binary)
81 changes: 66 additions & 15 deletions python/ray/includes/unique_ids.pxi
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ from ray.includes.unique_ids cimport (
CConfigID,
CDriverID,
CFunctionID,
CJobID,
CObjectID,
CTaskID,
CUniqueID,
Expand All @@ -45,11 +46,8 @@ cdef class UniqueID:
cdef CUniqueID data

def __init__(self, id):
if not id:
self.data = CUniqueID()
else:
check_id(id)
self.data = CUniqueID.from_binary(id)
check_id(id)
self.data = CUniqueID.from_binary(id)

@classmethod
def from_binary(cls, id_bytes):
Expand All @@ -59,7 +57,7 @@ cdef class UniqueID:

@classmethod
def nil(cls):
return cls(b"")
return cls(CUniqueID.nil().binary())

def __hash__(self):
return self.data.hash()
Expand Down Expand Up @@ -106,40 +104,93 @@ cdef class UniqueID:


cdef class ObjectID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CObjectID.from_binary(<c_string>id)

cdef CObjectID native(self):
return <CObjectID>self.data


cdef class TaskID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CTaskID.from_binary(<c_string>id)

cdef CTaskID native(self):
return <CTaskID>self.data


cdef class ClientID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CClientID.from_binary(<c_string>id)

cdef CClientID native(self):
return <CClientID>self.data


cdef class DriverID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CDriverID.from_binary(<c_string>id)

cdef CDriverID native(self):
return <CDriverID>self.data


cdef class ActorID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CActorID.from_binary(<c_string>id)

cdef CActorID native(self):
return <CActorID>self.data


cdef class ActorHandleID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CActorHandleID.from_binary(<c_string>id)

cdef CActorHandleID native(self):
return <CActorHandleID>self.data


cdef class ActorCheckpointID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CActorCheckpointID.from_binary(<c_string>id)

cdef CActorCheckpointID native(self):
return <CActorCheckpointID>self.data


cdef class FunctionID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CFunctionID.from_binary(<c_string>id)

cdef CFunctionID native(self):
return <CFunctionID>self.data


cdef class ActorClassID(UniqueID):
pass

def __init__(self, id):
check_id(id)
self.data = CActorClassID.from_binary(<c_string>id)

cdef CActorClassID native(self):
return <CActorClassID>self.data

_ID_TYPES = [
ActorCheckpointID,
Expand Down
Loading

0 comments on commit b9ea821

Please sign in to comment.