From 0b8d279ef263b8cd856ca033808e24c910ea2d1d Mon Sep 17 00:00:00 2001 From: Philipp Moritz Date: Sun, 5 Mar 2017 02:05:02 -0800 Subject: [PATCH] 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 --- build.sh | 7 +- src/common/CMakeLists.txt | 19 + src/common/common.h | 6 + src/common/common_protocol.cc | 26 + src/common/common_protocol.h | 41 ++ src/common/format/common.fbs | 52 ++ src/common/lib/python/common_extension.cc | 109 ++-- src/common/lib/python/common_extension.h | 13 +- src/common/redis_module/ray_redis_module.c | 6 +- src/common/state/local_scheduler_table.h | 4 +- src/common/state/redis.cc | 34 +- src/common/task.cc | 512 ++++++++---------- src/common/task.h | 175 +++--- src/common/test/db_tests.cc | 24 +- src/common/test/example_task.h | 59 ++ src/common/test/object_table_tests.cc | 7 +- src/common/test/task_table_tests.cc | 3 + src/common/test/task_tests.cc | 195 +++---- src/common/test/test_common.h | 50 +- src/global_scheduler/global_scheduler.cc | 8 +- .../global_scheduler_algorithm.cc | 32 +- .../global_scheduler_algorithm.h | 2 +- src/local_scheduler/CMakeLists.txt | 2 +- src/local_scheduler/local_scheduler.cc | 78 +-- src/local_scheduler/local_scheduler.h | 7 +- .../local_scheduler_algorithm.cc | 191 ++++--- .../local_scheduler_algorithm.h | 12 +- src/local_scheduler/local_scheduler_client.cc | 16 +- src/local_scheduler/local_scheduler_client.h | 7 +- .../local_scheduler_extension.cc | 13 +- src/local_scheduler/local_scheduler_shared.h | 4 +- .../test/local_scheduler_tests.cc | 132 +++-- src/plasma/CMakeLists.txt | 7 +- src/plasma/malloc.c | 1 - src/plasma/plasma_client.cc | 3 +- src/plasma/plasma_protocol.cc | 124 ++--- 36 files changed, 1052 insertions(+), 929 deletions(-) create mode 100644 src/common/common_protocol.cc create mode 100644 src/common/common_protocol.h create mode 100644 src/common/format/common.fbs create mode 100644 src/common/test/example_task.h diff --git a/build.sh b/build.sh index 8ce3e5fd927e9..a788bf7b390ca 100755 --- a/build.sh +++ b/build.sh @@ -25,7 +25,12 @@ bash "$ROOT_DIR/src/numbuf/thirdparty/build_thirdparty.sh" # Now build everything. pushd "$ROOT_DIR/python/ray/core" - cmake -DCMAKE_BUILD_TYPE=Release ../../.. + if [ "$VALGRIND" = "1" ] + then + cmake -DCMAKE_BUILD_TYPE=Debug ../../.. + else + cmake -DCMAKE_BUILD_TYPE=Release ../../.. + fi make clean make popd diff --git a/src/common/CMakeLists.txt b/src/common/CMakeLists.txt index e925497d7b375..ddb292a6dbb92 100644 --- a/src/common/CMakeLists.txt +++ b/src/common/CMakeLists.txt @@ -10,6 +10,22 @@ set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC") include_directories(thirdparty/ae) +# Compile flatbuffers + +set(COMMON_FBS_SRC "${CMAKE_CURRENT_LIST_DIR}/format/common.fbs") +set(OUTPUT_DIR ${CMAKE_CURRENT_LIST_DIR}/format/) + +add_custom_target(gen_common_fbs ALL) + +add_custom_command( + TARGET gen_common_fbs + COMMAND ${FLATBUFFERS_COMPILER} -c -o ${OUTPUT_DIR} ${COMMON_FBS_SRC} + DEPENDS ${FBS_DEPENDS} + COMMENT "Running flatc compiler on ${COMMON_FBS_SRC}" + VERBATIM) + +add_dependencies(gen_common_fbs flatbuffers_ep) + add_custom_target( hiredis COMMAND make @@ -18,6 +34,7 @@ add_custom_target( add_library(common STATIC event_loop.cc common.cc + common_protocol.cc task.cc io.cc net.cc @@ -32,6 +49,8 @@ add_library(common STATIC thirdparty/ae/ae.c thirdparty/sha256.c) +add_dependencies(common gen_common_fbs) + target_link_libraries(common "${CMAKE_CURRENT_LIST_DIR}/thirdparty/hiredis/libhiredis.a") function(define_test test_name library) diff --git a/src/common/common.h b/src/common/common.h index 01a17cf5bd7e1..7b04ea19b1f88 100644 --- a/src/common/common.h +++ b/src/common/common.h @@ -12,7 +12,13 @@ #endif #include "utarray.h" +#ifdef __cplusplus +extern "C" { +#endif #include "sha256.h" +#ifdef __cplusplus +} +#endif /** Definitions for Ray logging levels. */ #define RAY_COMMON_DEBUG 0 diff --git a/src/common/common_protocol.cc b/src/common/common_protocol.cc new file mode 100644 index 0000000000000..2787ce70f2023 --- /dev/null +++ b/src/common/common_protocol.cc @@ -0,0 +1,26 @@ +#include "common_protocol.h" + +flatbuffers::Offset to_flatbuf( + flatbuffers::FlatBufferBuilder &fbb, + ObjectID object_id) { + return fbb.CreateString((char *) &object_id.id[0], sizeof(object_id.id)); +} + +ObjectID from_flatbuf(const flatbuffers::String *string) { + ObjectID object_id; + CHECK(string->size() == sizeof(object_id.id)); + memcpy(&object_id.id[0], string->data(), sizeof(object_id.id)); + return object_id; +} + +flatbuffers::Offset< + flatbuffers::Vector>> +to_flatbuf(flatbuffers::FlatBufferBuilder &fbb, + ObjectID object_ids[], + int64_t num_objects) { + std::vector> results; + for (size_t i = 0; i < num_objects; i++) { + results.push_back(to_flatbuf(fbb, object_ids[i])); + } + return fbb.CreateVector(results); +} diff --git a/src/common/common_protocol.h b/src/common/common_protocol.h new file mode 100644 index 0000000000000..232dcd7ba44ca --- /dev/null +++ b/src/common/common_protocol.h @@ -0,0 +1,41 @@ +#ifndef COMMON_PROTOCOL_H +#define COMMON_PROTOCOL_H + +#include "format/common_generated.h" + +#include "common.h" + +/** + * Convert an object ID to a flatbuffer string. + * + * @param fbb Reference to the flatbuffer builder. + * @param object_id The object ID to be converted. + * @return The flatbuffer string contining the object ID. + */ +flatbuffers::Offset to_flatbuf( + flatbuffers::FlatBufferBuilder &fbb, + ObjectID object_id); + +/** + * Convert a flatbuffer string to an object ID. + * + * @param string The flatbuffer string. + * @return The object ID. + */ +ObjectID from_flatbuf(const flatbuffers::String *string); + +/** + * Convert an array of object IDs to a flatbuffer vector of strings. + * + * @param fbb Reference to the flatbuffer builder. + * @param object_ids Array of object IDs. + * @param num_objects Number of elements in the array. + * @return Flatbuffer vector of strings. + */ +flatbuffers::Offset< + flatbuffers::Vector>> +to_flatbuf(flatbuffers::FlatBufferBuilder &fbb, + ObjectID object_ids[], + int64_t num_objects); + +#endif diff --git a/src/common/format/common.fbs b/src/common/format/common.fbs new file mode 100644 index 0000000000000..f94ae879e472a --- /dev/null +++ b/src/common/format/common.fbs @@ -0,0 +1,52 @@ + +// Indices into resource vectors. +// A resource vector maps a resource index to the number +// of units of that resource required. + +// The total length of the resource vector is ResourceIndex_MAX. +enum ResourceIndex:int { + // A central processing unit. + CPU = 0, + // A graphics processing unit. + GPU = 1, + // A dummy entry to make ResourceIndex_MAX equal to the length of + // a resource vector. + DUMMY = 2 +} + +table Arg { + // Object ID for pass-by-reference arguments. + object_id: string; + // Data for pass-by-value arguments. + data: string; +} + +table TaskInfo { + // ID of the driver that created this task. + driver_id: string; + // Task ID of the task. + task_id: string; + // Task ID of the parent task. + parent_task_id: string; + // A count of the number of tasks submitted by the parent task before this one. + parent_counter: int; + // Actor ID of the task. This is the actor that this task is executed on + // or NIL_ACTOR_ID if the task is just a normal task. + actor_id: string; + // Number of tasks that have been submitted to this actor so far. + actor_counter: int; + // Function ID of the task. + function_id: string; + // Task arguments. + args: [Arg]; + // Object IDs of return values. + returns: [string]; + // The required_resources vector indicates the quantities of the different + // resources required by this task. The index in this vector corresponds to + // the resource type defined in the ResourceIndex enum. For example, + // required_resources[0] is the number of CPUs required, and + // required_resources[1] is the number of GPUs required. + required_resources: [double]; +} + +root_type TaskInfo; diff --git a/src/common/lib/python/common_extension.cc b/src/common/lib/python/common_extension.cc index 3faa2edcf5b62..a8c52ece1afe3 100644 --- a/src/common/lib/python/common_extension.cc +++ b/src/common/lib/python/common_extension.cc @@ -33,6 +33,8 @@ void init_pickle_module(void) { CHECK(pickle_protocol != NULL); } +TaskBuilder *g_task_builder = NULL; + /* Define the PyObjectID class. */ int PyStringToUniqueID(PyObject *object, ObjectID *object_id) { @@ -96,14 +98,10 @@ PyObject *PyTask_from_string(PyObject *self, PyObject *args) { } PyTask *result = PyObject_New(PyTask, &PyTaskType); result = (PyTask *) PyObject_Init((PyObject *) result, &PyTaskType); - result->spec = (task_spec *) malloc(size); + result->size = size; + result->spec = (TaskSpec *) malloc(size); memcpy(result->spec, data, size); - /* TODO(pcm): Better error checking once we use flatbuffers. */ - if (size != task_spec_size(result->spec)) { - PyErr_SetString(CommonError, - "task_from_string: task specification string malformed"); - return NULL; - } + /* TODO(pcm): Use flatbuffers validation here. */ return (PyObject *) result; } @@ -123,8 +121,7 @@ PyObject *PyTask_to_string(PyObject *self, PyObject *args) { return NULL; } PyTask *task = (PyTask *) arg; - return PyBytes_FromStringAndSize((char *) task->spec, - task_spec_size(task->spec)); + return PyBytes_FromStringAndSize((char *) task->spec, task->size); } static PyObject *PyObjectID_id(PyObject *self) { @@ -271,9 +268,7 @@ static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) { FunctionID function_id; /* Arguments of the task (can be PyObjectIDs or Python values). */ PyObject *arguments; - /* Array of pointers to string representations of pass-by-value args. */ - UT_array *val_repr_ptrs; - utarray_new(val_repr_ptrs, &ut_ptr_icd); + /* Number of return values of this task. */ int num_returns; /* The ID of the task that called this task. */ TaskID parent_task_id; @@ -289,101 +284,84 @@ static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) { return -1; } Py_ssize_t size = PyList_Size(arguments); - /* Determine the size of pass by value data in bytes. */ - Py_ssize_t value_data_bytes = 0; - for (Py_ssize_t i = 0; i < size; ++i) { - PyObject *arg = PyList_GetItem(arguments, i); - if (!PyObject_IsInstance(arg, (PyObject *) &PyObjectIDType)) { - CHECK(pickle_module != NULL); - CHECK(pickle_dumps != NULL); - PyObject *data = PyObject_CallMethodObjArgs(pickle_module, pickle_dumps, - arg, pickle_protocol, NULL); - value_data_bytes += PyBytes_Size(data); - utarray_push_back(val_repr_ptrs, &data); - } - } /* Construct the task specification. */ - int val_repr_index = 0; - self->spec = start_construct_task_spec( - driver_id, parent_task_id, parent_counter, actor_id, actor_counter, - function_id, size, num_returns, value_data_bytes); + TaskSpec_start_construct(g_task_builder, driver_id, parent_task_id, + parent_counter, actor_id, actor_counter, function_id, + num_returns); /* Add the task arguments. */ for (Py_ssize_t i = 0; i < size; ++i) { PyObject *arg = PyList_GetItem(arguments, i); if (PyObject_IsInstance(arg, (PyObject *) &PyObjectIDType)) { - task_args_add_ref(self->spec, ((PyObjectID *) arg)->object_id); + TaskSpec_args_add_ref(g_task_builder, ((PyObjectID *) arg)->object_id); } else { /* We do this check because we cast a signed int to an unsigned int. */ - CHECK(val_repr_index >= 0); - PyObject *data = *((PyObject **) utarray_eltptr( - val_repr_ptrs, (uint64_t) val_repr_index)); - task_args_add_val(self->spec, (uint8_t *) PyBytes_AS_STRING(data), - PyBytes_GET_SIZE(data)); + PyObject *data = PyObject_CallMethodObjArgs(pickle_module, pickle_dumps, + arg, pickle_protocol, NULL); + TaskSpec_args_add_val(g_task_builder, (uint8_t *) PyBytes_AS_STRING(data), + PyBytes_GET_SIZE(data)); Py_DECREF(data); - val_repr_index += 1; } } - utarray_free(val_repr_ptrs); /* Set the resource vector of the task. */ if (resource_vector != NULL) { - CHECK(PyList_Size(resource_vector) == MAX_RESOURCE_INDEX); - for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) { + CHECK(PyList_Size(resource_vector) == ResourceIndex_MAX); + for (int i = 0; i < ResourceIndex_MAX; ++i) { PyObject *resource_entry = PyList_GetItem(resource_vector, i); - task_spec_set_required_resource(self->spec, i, - PyFloat_AsDouble(resource_entry)); + TaskSpec_set_required_resource(g_task_builder, i, + PyFloat_AsDouble(resource_entry)); } } else { - for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) { - task_spec_set_required_resource(self->spec, i, - i == CPU_RESOURCE_INDEX ? 1.0 : 0.0); + for (int i = 0; i < ResourceIndex_MAX; ++i) { + TaskSpec_set_required_resource(g_task_builder, i, + i == ResourceIndex_CPU ? 1.0 : 0.0); } } /* Compute the task ID and the return object IDs. */ - finish_construct_task_spec(self->spec); + self->spec = TaskSpec_finish_construct(g_task_builder, &self->size); return 0; } static void PyTask_dealloc(PyTask *self) { if (self->spec != NULL) { - free_task_spec(self->spec); + TaskSpec_free(self->spec); } Py_TYPE(self)->tp_free((PyObject *) self); } static PyObject *PyTask_function_id(PyObject *self) { - FunctionID function_id = task_function(((PyTask *) self)->spec); + FunctionID function_id = TaskSpec_function(((PyTask *) self)->spec); return PyObjectID_make(function_id); } static PyObject *PyTask_actor_id(PyObject *self) { - ActorID actor_id = task_spec_actor_id(((PyTask *) self)->spec); + ActorID actor_id = TaskSpec_actor_id(((PyTask *) self)->spec); return PyObjectID_make(actor_id); } static PyObject *PyTask_driver_id(PyObject *self) { - UniqueID driver_id = task_spec_driver_id(((PyTask *) self)->spec); + UniqueID driver_id = TaskSpec_driver_id(((PyTask *) self)->spec); return PyObjectID_make(driver_id); } static PyObject *PyTask_task_id(PyObject *self) { - TaskID task_id = task_spec_id(((PyTask *) self)->spec); + TaskID task_id = TaskSpec_task_id(((PyTask *) self)->spec); return PyObjectID_make(task_id); } static PyObject *PyTask_arguments(PyObject *self) { - task_spec *task = ((PyTask *) self)->spec; - int64_t num_args = task_num_args(task); + TaskSpec *task = ((PyTask *) self)->spec; + int64_t num_args = TaskSpec_num_args(task); PyObject *arg_list = PyList_New((Py_ssize_t) num_args); for (int i = 0; i < num_args; ++i) { - if (task_arg_type(task, i) == ARG_BY_REF) { - ObjectID object_id = task_arg_id(task, i); + if (TaskSpec_arg_by_ref(task, i)) { + ObjectID object_id = TaskSpec_arg_id(task, i); PyList_SetItem(arg_list, i, PyObjectID_make(object_id)); } else { CHECK(pickle_module != NULL); CHECK(pickle_loads != NULL); PyObject *str = - PyBytes_FromStringAndSize((char *) task_arg_val(task, i), - (Py_ssize_t) task_arg_length(task, i)); + PyBytes_FromStringAndSize((char *) TaskSpec_arg_val(task, i), + (Py_ssize_t) TaskSpec_arg_length(task, i)); PyObject *val = PyObject_CallMethodObjArgs(pickle_module, pickle_loads, str, NULL); Py_XDECREF(str); @@ -394,21 +372,21 @@ static PyObject *PyTask_arguments(PyObject *self) { } static PyObject *PyTask_required_resources(PyObject *self) { - task_spec *task = ((PyTask *) self)->spec; - PyObject *required_resources = PyList_New((Py_ssize_t) MAX_RESOURCE_INDEX); - for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) { - double r = task_spec_get_required_resource(task, i); + TaskSpec *task = ((PyTask *) self)->spec; + PyObject *required_resources = PyList_New((Py_ssize_t) ResourceIndex_MAX); + for (int i = 0; i < ResourceIndex_MAX; ++i) { + double r = TaskSpec_get_required_resource(task, i); PyList_SetItem(required_resources, i, PyFloat_FromDouble(r)); } return required_resources; } static PyObject *PyTask_returns(PyObject *self) { - task_spec *task = ((PyTask *) self)->spec; - int64_t num_returns = task_num_returns(task); + TaskSpec *task = ((PyTask *) self)->spec; + int64_t num_returns = TaskSpec_num_returns(task); PyObject *return_id_list = PyList_New((Py_ssize_t) num_returns); for (int i = 0; i < num_returns; ++i) { - ObjectID object_id = task_return(task, i); + ObjectID object_id = TaskSpec_return(task, i); PyList_SetItem(return_id_list, i, PyObjectID_make(object_id)); } return return_id_list; @@ -474,11 +452,12 @@ PyTypeObject PyTaskType = { }; /* Create a PyTask from a C struct. The resulting PyTask takes ownership of the - * task_spec and will deallocate the task_spec in the PyTask destructor. */ -PyObject *PyTask_make(task_spec *task_spec) { + * TaskSpec and will deallocate the TaskSpec in the PyTask destructor. */ +PyObject *PyTask_make(TaskSpec *task_spec, int64_t task_size) { PyTask *result = PyObject_New(PyTask, &PyTaskType); result = (PyTask *) PyObject_Init((PyObject *) result, &PyTaskType); result->spec = task_spec; + result->size = task_size; return (PyObject *) result; } diff --git a/src/common/lib/python/common_extension.h b/src/common/lib/python/common_extension.h index a2ea17b3ecd24..c91b631cd3308 100644 --- a/src/common/lib/python/common_extension.h +++ b/src/common/lib/python/common_extension.h @@ -6,7 +6,9 @@ #include "structmember.h" #include "common.h" -#include "task.h" + +typedef uint8_t TaskSpec; +struct TaskBuilder; extern PyObject *CommonError; @@ -18,7 +20,8 @@ typedef struct { typedef struct { PyObject_HEAD - task_spec *spec; + int64_t size; + TaskSpec *spec; } PyTask; // clang-format on @@ -33,9 +36,11 @@ extern PyObject *pickle_loads; void init_pickle_module(void); +extern TaskBuilder *g_task_builder; + int PyStringToUniqueID(PyObject *object, ObjectID *object_id); -int PyObjectToUniqueID(PyObject *object, ObjectID *objectid); +int PyObjectToUniqueID(PyObject *object, ObjectID *object_id); PyObject *PyObjectID_make(ObjectID object_id); @@ -46,6 +51,6 @@ PyObject *PyTask_from_string(PyObject *, PyObject *args); PyObject *compute_put_id(PyObject *self, PyObject *args); -PyObject *PyTask_make(task_spec *task_spec); +PyObject *PyTask_make(TaskSpec *task_spec, int64_t task_size); #endif /* COMMON_EXTENSION_H */ diff --git a/src/common/redis_module/ray_redis_module.c b/src/common/redis_module/ray_redis_module.c index 8a940d9f1e14a..a606cd6828d4a 100644 --- a/src/common/redis_module/ray_redis_module.c +++ b/src/common/redis_module/ray_redis_module.c @@ -714,7 +714,7 @@ int ReplyWithTask(RedisModuleCtx *ctx, RedisModuleString *task_id) { RedisModuleString *local_scheduler_id = NULL; RedisModuleString *task_spec = NULL; RedisModule_HashGet(key, REDISMODULE_HASH_CFIELDS, "state", &state, "node", - &local_scheduler_id, "task_spec", &task_spec, NULL); + &local_scheduler_id, "TaskSpec", &task_spec, NULL); if (state == NULL || local_scheduler_id == NULL || task_spec == NULL) { /* We must have either all fields or no fields. */ RedisModule_CloseKey(key); @@ -817,7 +817,7 @@ int TaskTableWrite(RedisModuleCtx *ctx, if (task_spec == NULL) { RedisModule_HashSet(key, REDISMODULE_HASH_CFIELDS, "state", state, "node", node_id, NULL); - RedisModule_HashGet(key, REDISMODULE_HASH_CFIELDS, "task_spec", + RedisModule_HashGet(key, REDISMODULE_HASH_CFIELDS, "TaskSpec", &existing_task_spec, NULL); if (existing_task_spec == NULL) { RedisModule_CloseKey(key); @@ -827,7 +827,7 @@ int TaskTableWrite(RedisModuleCtx *ctx, } } else { RedisModule_HashSet(key, REDISMODULE_HASH_CFIELDS, "state", state, "node", - node_id, "task_spec", task_spec, NULL); + node_id, "TaskSpec", task_spec, NULL); } RedisModule_CloseKey(key); diff --git a/src/common/state/local_scheduler_table.h b/src/common/state/local_scheduler_table.h index a0eb09fce5d2a..8228fa4fb2d1e 100644 --- a/src/common/state/local_scheduler_table.h +++ b/src/common/state/local_scheduler_table.h @@ -17,10 +17,10 @@ typedef struct { int available_workers; /** The resource vector of resources generally available to this local * scheduler. */ - double static_resources[MAX_RESOURCE_INDEX]; + double static_resources[ResourceIndex_MAX]; /** The resource vector of resources currently available to this local * scheduler. */ - double dynamic_resources[MAX_RESOURCE_INDEX]; + double dynamic_resources[ResourceIndex_MAX]; } LocalSchedulerInfo; /* diff --git a/src/common/state/redis.cc b/src/common/state/redis.cc index 3e3662dd7e693..31b6d82e00fc2 100644 --- a/src/common/state/redis.cc +++ b/src/common/state/redis.cc @@ -358,7 +358,7 @@ Task *parse_and_construct_task_from_redis_reply(redisReply *reply) { } else if (reply->type == REDIS_REPLY_ARRAY) { /* Check that the reply is as expected. The 0th element is the scheduling * state. The 1st element is the db_client_id of the associated local - * scheduler, and the 2nd element is the task_spec. */ + * scheduler, and the 2nd element is the TaskSpec. */ CHECK(reply->elements == 3); CHECK(reply->element[0]->type == REDIS_REPLY_INTEGER); CHECK(reply->element[1]->type == REDIS_REPLY_STRING); @@ -371,12 +371,11 @@ Task *parse_and_construct_task_from_redis_reply(redisReply *reply) { memcpy(local_scheduler_id.id, reply->element[1]->str, reply->element[1]->len); /* Parse the task spec. */ - task_spec *spec = (task_spec *) malloc(reply->element[2]->len); + TaskSpec *spec = (TaskSpec *) malloc(reply->element[2]->len); memcpy(spec, reply->element[2]->str, reply->element[2]->len); - CHECK(task_spec_size(spec) == reply->element[2]->len); - task = Task_alloc(spec, state, local_scheduler_id); + task = Task_alloc(spec, reply->element[2]->len, state, local_scheduler_id); /* Free the task spec. */ - free_task_spec(spec); + TaskSpec_free(spec); } else { LOG_FATAL("Unexpected reply type %d", reply->type); } @@ -777,16 +776,16 @@ void redis_task_table_add_task(TableCallbackData *callback_data) { DBHandle *db = callback_data->db_handle; Task *task = (Task *) callback_data->data; TaskID task_id = Task_task_id(task); - DBClientID local_scheduler_id = Task_local_scheduler_id(task); + DBClientID local_scheduler_id = Task_local_scheduler(task); int state = Task_state(task); - task_spec *spec = Task_task_spec(task); + TaskSpec *spec = Task_task_spec(task); CHECKM(task != NULL, "NULL task passed to redis_task_table_add_task."); int status = redisAsyncCommand( db->context, redis_task_table_add_task_callback, (void *) callback_data->timer_id, "RAY.TASK_TABLE_ADD %b %d %b %b", task_id.id, sizeof(task_id.id), state, local_scheduler_id.id, - sizeof(local_scheduler_id.id), spec, task_spec_size(spec)); + sizeof(local_scheduler_id.id), spec, Task_task_spec_size(task)); if ((status == REDIS_ERR) || db->context->err) { LOG_REDIS_DEBUG(db->context, "error in redis_task_table_add_task"); } @@ -814,7 +813,7 @@ void redis_task_table_update(TableCallbackData *callback_data) { DBHandle *db = callback_data->db_handle; Task *task = (Task *) callback_data->data; TaskID task_id = Task_task_id(task); - DBClientID local_scheduler_id = Task_local_scheduler_id(task); + DBClientID local_scheduler_id = Task_local_scheduler(task); int state = Task_state(task); CHECKM(task != NULL, "NULL task passed to redis_task_table_update."); @@ -875,7 +874,8 @@ void parse_task_table_subscribe_callback(char *payload, TaskID *task_id, int *state, DBClientID *local_scheduler_id, - task_spec **spec) { + TaskSpec **spec, + int64_t *task_spec_size) { /* Note that the state is padded with spaces to consist of precisely two * characters. */ int task_spec_payload_size = @@ -902,9 +902,9 @@ void parse_task_table_subscribe_callback(char *payload, CHECK(memcmp(space_str, &payload[offset], strlen(space_str)) == 0); offset += strlen(space_str); /* Read in the task spec. */ - *spec = (task_spec *) malloc(task_spec_payload_size); + *spec = (TaskSpec *) malloc(task_spec_payload_size); memcpy(*spec, &payload[offset], task_spec_payload_size); - CHECK(task_spec_size(*spec) == task_spec_payload_size); + *task_spec_size = task_spec_payload_size; } void redis_task_table_subscribe_callback(redisAsyncContext *c, @@ -932,11 +932,13 @@ void redis_task_table_subscribe_callback(redisAsyncContext *c, TaskID task_id; int state; DBClientID local_scheduler_id; - task_spec *spec; + TaskSpec *spec; + int64_t task_spec_size; parse_task_table_subscribe_callback(payload->str, payload->len, &task_id, - &state, &local_scheduler_id, &spec); - Task *task = Task_alloc(spec, state, local_scheduler_id); - free(spec); + &state, &local_scheduler_id, &spec, + &task_spec_size); + Task *task = Task_alloc(spec, task_spec_size, state, local_scheduler_id); + TaskSpec_free(spec); /* Call the subscribe callback if there is one. */ if (data->subscribe_callback != NULL) { data->subscribe_callback(task, data->subscribe_context); diff --git a/src/common/task.cc b/src/common/task.cc index 9b7f8bd2a47e2..9fe9697819d08 100644 --- a/src/common/task.cc +++ b/src/common/task.cc @@ -1,134 +1,11 @@ -#include -#include -#include +#include -extern "C" { -#include "sha256.h" -} -#include "utarray.h" +#include "common_protocol.h" #include "task.h" -#include "common.h" -#include "io.h" - -/* TASK SPECIFICATIONS */ - -/* Tasks are stored in a consecutive chunk of memory, the first - * sizeof(task_spec) bytes are arranged according to the struct - * task_spec. Then there is an array of task_args of length - * (num_args + num_returns), and then follows the data of - * pass-by-value arguments of size args_value_size. The offsets in the - * task_arg.val are with respect to the end of the augmented structure, - * i.e. with respect to the address &task_spec.args_and_returns[0] + - * (task_spec->num_args + task_spec->num_returns) * sizeof(task_arg). */ - -typedef struct { - /* Either ARG_BY_REF or ARG_BY_VAL. */ - int8_t type; - union { - ObjectID obj_id; - struct { - /* Offset where the data associated to this arg is located relative - * to &task_spec.args_and_returns[0]. */ - ptrdiff_t offset; - int64_t length; - } value; - }; -} task_arg; - -struct task_spec_impl { - /** ID of the driver that created this task. */ - UniqueID driver_id; - /** Task ID of the task. */ - TaskID task_id; - /** Task ID of the parent task. */ - TaskID parent_task_id; - /** A count of the number of tasks submitted by the parent task before this - * one. */ - int64_t parent_counter; - /** Actor ID of the task. This is the actor that this task is executed on - * or NIL_ACTOR_ID if the task is just a normal task. */ - ActorID actor_id; - /** Number of tasks that have been submitted to this actor so far. */ - int64_t actor_counter; - /** Function ID of the task. */ - FunctionID function_id; - /** Total number of arguments. */ - int64_t num_args; - /** Index of the last argument that has been constructed. */ - int64_t arg_index; - /** Number of return values. */ - int64_t num_returns; - /** Number of bytes the pass-by-value arguments are occupying. */ - int64_t args_value_size; - /** The offset of the number of bytes of pass-by-value data that - * has been written so far, relative to &task_spec->args_and_returns[0] + - * (task_spec->num_args + task_spec->num_returns) * sizeof(task_arg) */ - int64_t args_value_offset; - /** Resource vector for this task. A resource vector maps a resource index - * (like "cpu" or "gpu") to the number of units of that resource required. - * Note that this will allow us to support arbitrary attributes: - * For example, we can have a coloring of nodes and "red" can correspond - * to 0.0, "green" to 1.0 and "yellow" to 2.0. */ - double required_resources[MAX_RESOURCE_INDEX]; - /** Argument and return IDs as well as offsets for pass-by-value args. */ - task_arg args_and_returns[0]; -}; - -/* The size of a task specification is given by the following expression. */ -#define TASK_SPEC_SIZE(NUM_ARGS, NUM_RETURNS, ARGS_VALUE_SIZE) \ - (sizeof(task_spec) + ((NUM_ARGS) + (NUM_RETURNS)) * sizeof(task_arg) + \ - (ARGS_VALUE_SIZE)) - -bool TaskID_equal(TaskID first_id, TaskID second_id) { - return UNIQUE_ID_EQ(first_id, second_id); -} - -bool TaskID_is_nil(TaskID id) { - return TaskID_equal(id, NIL_TASK_ID); -} - -bool ActorID_equal(ActorID first_id, ActorID second_id) { - return UNIQUE_ID_EQ(first_id, second_id); -} - -bool FunctionID_equal(FunctionID first_id, FunctionID second_id) { - return UNIQUE_ID_EQ(first_id, second_id); -} - -bool FunctionID_is_nil(FunctionID id) { - return FunctionID_equal(id, NIL_FUNCTION_ID); -} - -TaskID *task_return_ptr(task_spec *spec, int64_t return_index) { - DCHECK(0 <= return_index && return_index < spec->num_returns); - task_arg *ret = &spec->args_and_returns[spec->num_args + return_index]; - DCHECK(ret->type == ARG_BY_REF); - return &ret->obj_id; -} -/* Compute the task ID. This assumes that all of the other fields have been set - * and that the return IDs have not been set. It assumes the task_spec was - * zero-initialized so that uninitialized fields will not make the task ID - * nondeterministic. */ -TaskID compute_task_id(task_spec *spec) { - /* Check that the task ID and return ID fields of the task_spec are - * uninitialized. */ - DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID)); - for (int i = 0; i < spec->num_returns; ++i) { - DCHECK(ObjectID_equal(*task_return_ptr(spec, i), NIL_ID)); - } - /* Compute a SHA256 hash of the task_spec. */ - SHA256_CTX ctx; - BYTE buff[DIGEST_SIZE]; - sha256_init(&ctx); - sha256_update(&ctx, (BYTE *) spec, task_spec_size(spec)); - sha256_final(&ctx, buff); - /* Create a task ID out of the hash. This will truncate the hash. */ - TaskID task_id; - CHECK(sizeof(task_id) <= DIGEST_SIZE); - memcpy(&task_id.id, buff, sizeof(task_id.id)); - return task_id; +extern "C" { +#include "sha256.h" } ObjectID task_compute_return_id(TaskID task_id, int64_t return_index) { @@ -155,209 +32,266 @@ ObjectID task_compute_put_id(TaskID task_id, int64_t put_index) { return put_id; } -task_spec *start_construct_task_spec(UniqueID driver_id, - TaskID parent_task_id, - int64_t parent_counter, - ActorID actor_id, - int64_t actor_counter, - FunctionID function_id, - int64_t num_args, - int64_t num_returns, - int64_t args_value_size) { - int64_t size = TASK_SPEC_SIZE(num_args, num_returns, args_value_size); - task_spec *task = (task_spec *) malloc(size); - memset(task, 0, size); - task->driver_id = driver_id; - task->task_id = NIL_TASK_ID; - task->parent_task_id = parent_task_id; - task->parent_counter = parent_counter; - task->actor_id = actor_id; - task->actor_counter = actor_counter; - task->function_id = function_id; - task->num_args = num_args; - task->arg_index = 0; - task->num_returns = num_returns; - task->args_value_size = args_value_size; - for (int i = 0; i < num_returns; ++i) { - *task_return_ptr(task, i) = NIL_ID; +class TaskBuilder { + public: + void Start(UniqueID driver_id, + TaskID parent_task_id, + int64_t parent_counter, + ActorID actor_id, + int64_t actor_counter, + FunctionID function_id, + int64_t num_returns) { + driver_id_ = driver_id; + parent_task_id_ = parent_task_id; + parent_counter_ = parent_counter; + actor_id_ = actor_id; + actor_counter_ = actor_counter; + function_id_ = function_id; + num_returns_ = num_returns; + + /* Compute hashes. */ + sha256_init(&ctx); + sha256_update(&ctx, (BYTE *) &driver_id, sizeof(driver_id)); + sha256_update(&ctx, (BYTE *) &parent_task_id, sizeof(parent_task_id)); + sha256_update(&ctx, (BYTE *) &parent_counter, sizeof(parent_counter)); + sha256_update(&ctx, (BYTE *) &actor_id, sizeof(actor_id)); + sha256_update(&ctx, (BYTE *) &actor_counter, sizeof(actor_counter)); + sha256_update(&ctx, (BYTE *) &function_id, sizeof(function_id)); + } + + void NextReferenceArgument(ObjectID object_id) { + args.push_back(CreateArg(fbb, to_flatbuf(fbb, object_id))); + sha256_update(&ctx, (BYTE *) &object_id, sizeof(object_id)); + } + + void NextValueArgument(uint8_t *value, int64_t length) { + auto arg = fbb.CreateString((const char *) value, length); + auto empty_id = fbb.CreateString("", 0); + args.push_back(CreateArg(fbb, empty_id, arg)); + sha256_update(&ctx, (BYTE *) &value, length); + } + + void SetRequiredResource(int64_t resource_index, double value) { + if (resource_index >= resource_vector_.size()) { + /* Make sure the resource vector is constructed entry by entry, + * in order. */ + CHECK(resource_index == resource_vector_.size()); + resource_vector_.resize(resource_index + 1); + } + resource_vector_[resource_index] = value; } - return task; -} -void finish_construct_task_spec(task_spec *spec) { - /* Check that all of the arguments were added to the task. */ - DCHECK(spec->arg_index == spec->num_args); - spec->task_id = compute_task_id(spec); - /* Set the object IDs for the return values. */ - for (int64_t i = 0; i < spec->num_returns; ++i) { - *task_return_ptr(spec, i) = task_compute_return_id(spec->task_id, i); + uint8_t *Finish(int64_t *size) { + /* Add arguments. */ + auto arguments = fbb.CreateVector(args); + /* Update hash. */ + BYTE buff[DIGEST_SIZE]; + sha256_final(&ctx, buff); + TaskID task_id; + CHECK(sizeof(task_id) <= DIGEST_SIZE); + memcpy(&task_id, buff, sizeof(task_id)); + /* Add return object IDs. */ + std::vector> returns; + for (int64_t i = 0; i < num_returns_; i++) { + ObjectID return_id = task_compute_return_id(task_id, i); + returns.push_back(to_flatbuf(fbb, return_id)); + } + /* Create TaskInfo. */ + for (int64_t i = resource_vector_.size(); i < ResourceIndex_MAX; ++i) { + resource_vector_.push_back(0.0); + } + auto message = CreateTaskInfo( + fbb, to_flatbuf(fbb, driver_id_), to_flatbuf(fbb, task_id), + to_flatbuf(fbb, parent_task_id_), parent_counter_, + to_flatbuf(fbb, actor_id_), actor_counter_, + to_flatbuf(fbb, function_id_), arguments, fbb.CreateVector(returns), + fbb.CreateVector(resource_vector_)); + /* Finish the TaskInfo. */ + fbb.Finish(message); + *size = fbb.GetSize(); + uint8_t *result = (uint8_t *) malloc(*size); + memcpy(result, fbb.GetBufferPointer(), *size); + fbb.Clear(); + args.clear(); + return result; } + + private: + flatbuffers::FlatBufferBuilder fbb; + std::vector> args; + SHA256_CTX ctx; + + /* Data for the builder. */ + UniqueID driver_id_; + TaskID parent_task_id_; + int64_t parent_counter_; + ActorID actor_id_; + int64_t actor_counter_; + FunctionID function_id_; + int64_t num_returns_; + std::vector resource_vector_; +}; + +TaskBuilder *make_task_builder(void) { + return new TaskBuilder(); } -int64_t task_spec_size(task_spec *spec) { - return TASK_SPEC_SIZE(spec->num_args, spec->num_returns, - spec->args_value_size); +void free_task_builder(TaskBuilder *builder) { + delete builder; } -FunctionID task_function(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - return spec->function_id; +bool TaskID_equal(TaskID first_id, TaskID second_id) { + return UNIQUE_ID_EQ(first_id, second_id); } -ActorID task_spec_actor_id(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - return spec->actor_id; +bool TaskID_is_nil(TaskID id) { + return TaskID_equal(id, NIL_TASK_ID); } -int64_t task_spec_actor_counter(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - return spec->actor_counter; +bool ActorID_equal(ActorID first_id, ActorID second_id) { + return UNIQUE_ID_EQ(first_id, second_id); } -UniqueID task_spec_driver_id(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - return spec->driver_id; +bool FunctionID_equal(FunctionID first_id, FunctionID second_id) { + return UNIQUE_ID_EQ(first_id, second_id); } -TaskID task_spec_id(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - return spec->task_id; +bool FunctionID_is_nil(FunctionID id) { + return FunctionID_equal(id, NIL_FUNCTION_ID); } -int64_t task_num_args(task_spec *spec) { - return spec->num_args; +/* Functions for building tasks. */ + +void TaskSpec_start_construct(TaskBuilder *builder, + UniqueID driver_id, + TaskID parent_task_id, + int64_t parent_counter, + ActorID actor_id, + int64_t actor_counter, + FunctionID function_id, + int64_t num_returns) { + builder->Start(driver_id, parent_task_id, parent_counter, actor_id, + actor_counter, function_id, num_returns); } -int64_t task_num_returns(task_spec *spec) { - return spec->num_returns; +uint8_t *TaskSpec_finish_construct(TaskBuilder *builder, int64_t *size) { + return builder->Finish(size); } -int8_t task_arg_type(task_spec *spec, int64_t arg_index) { - DCHECK(0 <= arg_index && arg_index < spec->num_args); - return spec->args_and_returns[arg_index].type; +void TaskSpec_args_add_ref(TaskBuilder *builder, ObjectID object_id) { + builder->NextReferenceArgument(object_id); } -ObjectID task_arg_id(task_spec *spec, int64_t arg_index) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - DCHECK(0 <= arg_index && arg_index < spec->num_args); - task_arg *arg = &spec->args_and_returns[arg_index]; - DCHECK(arg->type == ARG_BY_REF) - return arg->obj_id; +void TaskSpec_args_add_val(TaskBuilder *builder, + uint8_t *value, + int64_t length) { + builder->NextValueArgument(value, length); } -uint8_t *task_arg_val(task_spec *spec, int64_t arg_index) { - DCHECK(0 <= arg_index && arg_index < spec->num_args); - task_arg *arg = &spec->args_and_returns[arg_index]; - DCHECK(arg->type == ARG_BY_VAL); - uint8_t *data = (uint8_t *) &spec->args_and_returns[0]; - data += (spec->num_args + spec->num_returns) * sizeof(task_arg); - return data + arg->value.offset; +void TaskSpec_set_required_resource(TaskBuilder *builder, + int64_t resource_index, + double value) { + builder->SetRequiredResource(resource_index, value); } -int64_t task_arg_length(task_spec *spec, int64_t arg_index) { - DCHECK(0 <= arg_index && arg_index < spec->num_args); - task_arg *arg = &spec->args_and_returns[arg_index]; - DCHECK(arg->type == ARG_BY_VAL); - return arg->value.length; +/* Functions for reading tasks. */ + +TaskID TaskSpec_task_id(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->task_id()); } -int64_t task_args_add_ref(task_spec *spec, ObjectID obj_id) { - /* Check that the task is still under construction. */ - DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID)); - task_arg *arg = &spec->args_and_returns[spec->arg_index]; - arg->type = ARG_BY_REF; - arg->obj_id = obj_id; - return spec->arg_index++; +FunctionID TaskSpec_function(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->function_id()); } -int64_t task_args_add_val(task_spec *spec, uint8_t *data, int64_t length) { - /* Check that the task is still under construction. */ - DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID)); - task_arg *arg = &spec->args_and_returns[spec->arg_index]; - arg->type = ARG_BY_VAL; - arg->value.offset = spec->args_value_offset; - arg->value.length = length; - uint8_t *addr = task_arg_val(spec, spec->arg_index); - DCHECK(spec->args_value_offset + length <= spec->args_value_size); - DCHECK(spec->arg_index != spec->num_args - 1 || - spec->args_value_offset + length == spec->args_value_size); - memcpy(addr, data, length); - spec->args_value_offset += length; - return spec->arg_index++; +ActorID TaskSpec_actor_id(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->actor_id()); } -void task_spec_set_required_resource(task_spec *spec, - int64_t resource_index, - double value) { - spec->required_resources[resource_index] = value; +int64_t TaskSpec_actor_counter(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->actor_counter(); } -ObjectID task_return(task_spec *spec, int64_t return_index) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - DCHECK(0 <= return_index && return_index < spec->num_returns); - task_arg *ret = &spec->args_and_returns[spec->num_args + return_index]; - DCHECK(ret->type == ARG_BY_REF); - return ret->obj_id; +UniqueID TaskSpec_driver_id(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->driver_id()); } -double task_spec_get_required_resource(const task_spec *spec, - int64_t resource_index) { - return spec->required_resources[resource_index]; +int64_t TaskSpec_num_args(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->args()->size(); } -void free_task_spec(task_spec *spec) { - /* Check that the task has been constructed. */ - DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID)); - DCHECK(spec->arg_index == spec->num_args); - free(spec); +ObjectID TaskSpec_arg_id(TaskSpec *spec, int64_t arg_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->args()->Get(arg_index)->object_id()); } -void print_task(task_spec *spec, UT_string *output) { - /* For converting an id to hex, which has double the number - * of bytes compared to the id (+ 1 byte for '\0'). */ - static char hex[ID_STRING_SIZE]; - /* Print function id. */ - ObjectID_to_string((ObjectID) task_function(spec), &hex[0], ID_STRING_SIZE); - utstring_printf(output, "fun %s ", &hex[0]); - /* Print arguments. */ - for (int i = 0; i < task_num_args(spec); ++i) { - ObjectID_to_string((ObjectID) task_arg_id(spec, i), &hex[0], - ID_STRING_SIZE); - utstring_printf(output, " id:%d %s", i, &hex[0]); - } - /* Print return ids. */ - for (int i = 0; i < task_num_returns(spec); ++i) { - ObjectID obj_id = task_return(spec, i); - ObjectID_to_string(obj_id, &hex[0], ID_STRING_SIZE); - utstring_printf(output, " ret:%d %s", i, &hex[0]); - } +const uint8_t *TaskSpec_arg_val(TaskSpec *spec, int64_t arg_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return (uint8_t *) message->args()->Get(arg_index)->data()->c_str(); } -/* TASK INSTANCES */ +int64_t TaskSpec_arg_length(TaskSpec *spec, int64_t arg_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->args()->Get(arg_index)->data()->size(); +} -struct TaskImpl { - /** The scheduling state of the task. */ - int state; - /** The ID of the local scheduler involved. */ - DBClientID local_scheduler_id; - /** The task specification for this task. */ - task_spec spec; -}; +int64_t TaskSpec_num_returns(TaskSpec *spec) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->returns()->size(); +} + +bool TaskSpec_arg_by_ref(TaskSpec *spec, int64_t arg_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->args()->Get(arg_index)->object_id()->size() != 0; +} + +ObjectID TaskSpec_return(TaskSpec *spec, int64_t return_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return from_flatbuf(message->returns()->Get(return_index)); +} -Task *Task_alloc(task_spec *spec, int state, DBClientID local_scheduler_id) { - int64_t size = sizeof(Task) - sizeof(task_spec) + task_spec_size(spec); +double TaskSpec_get_required_resource(const TaskSpec *spec, + int64_t resource_index) { + CHECK(spec); + auto message = flatbuffers::GetRoot(spec); + return message->required_resources()->Get(resource_index); +} + +void TaskSpec_free(TaskSpec *spec) { + free(spec); +} + +/* TASK INSTANCES */ + +Task *Task_alloc(TaskSpec *spec, + int64_t task_spec_size, + int state, + DBClientID local_scheduler_id) { + int64_t size = sizeof(Task) - sizeof(TaskSpec) + task_spec_size; Task *result = (Task *) malloc(size); memset(result, 0, size); result->state = state; result->local_scheduler_id = local_scheduler_id; - memcpy(&result->spec, spec, task_spec_size(spec)); + result->task_spec_size = task_spec_size; + memcpy(&result->spec, spec, task_spec_size); return result; } @@ -370,7 +304,7 @@ Task *Task_copy(Task *other) { } int64_t Task_size(Task *task_arg) { - return sizeof(Task) - sizeof(task_spec) + task_spec_size(&task_arg->spec); + return sizeof(Task) - sizeof(TaskSpec) + task_arg->task_spec_size; } int Task_state(Task *task) { @@ -381,21 +315,25 @@ void Task_set_state(Task *task, int state) { task->state = state; } -DBClientID Task_local_scheduler_id(Task *task) { +DBClientID Task_local_scheduler(Task *task) { return task->local_scheduler_id; } -void Task_set_local_scheduler_id(Task *task, DBClientID local_scheduler_id) { +void Task_set_local_scheduler(Task *task, DBClientID local_scheduler_id) { task->local_scheduler_id = local_scheduler_id; } -task_spec *Task_task_spec(Task *task) { +TaskSpec *Task_task_spec(Task *task) { return &task->spec; } +int64_t Task_task_spec_size(Task *task) { + return task->task_spec_size; +} + TaskID Task_task_id(Task *task) { - task_spec *spec = Task_task_spec(task); - return task_spec_id(spec); + TaskSpec *spec = Task_task_spec(task); + return TaskSpec_task_id(spec); } void Task_free(Task *task) { diff --git a/src/common/task.h b/src/common/task.h index edcff973672d7..a25728158fb88 100644 --- a/src/common/task.h +++ b/src/common/task.h @@ -1,19 +1,18 @@ #ifndef TASK_H #define TASK_H -/** - * This API specifies the task data structures. It is in C so we can - * easily construct tasks from other languages like Python. The data structures - * are also defined in such a way that memory is contiguous and all pointers - * are relative, so that we can memcpy the datastructure and ship it over the - * network without serialization and deserialization. */ - #include #include #include #include "common.h" #include "utstring.h" +#include "format/common_generated.h" + +typedef uint8_t TaskSpec; + +struct TaskBuilder; + #define NIL_TASK_ID NIL_ID #define NIL_ACTOR_ID NIL_ID #define NIL_FUNCTION_ID NIL_ID @@ -28,17 +27,6 @@ typedef UniqueID TaskID; * not run on an actor, then NIL_ACTOR_ID should be used. */ typedef UniqueID ActorID; -/** - * ==== Task specifications ==== - * Contain all the information neccessary to execute the - * task (function id, arguments, return object ids). - */ - -typedef struct task_spec_impl task_spec; - -/** If argument is passed by value or reference. */ -enum arg_type { ARG_BY_REF, ARG_BY_VAL }; - /** * Compare two task IDs. * @@ -84,6 +72,10 @@ bool FunctionID_is_nil(FunctionID id); /* Construct and modify task specifications. */ +TaskBuilder *make_task_builder(void); + +void free_task_builder(TaskBuilder *builder); + /** * Begin constructing a task_spec. After this is called, the arguments must be * added to the task_spec and then finish_construct_task_spec must be called. @@ -93,8 +85,6 @@ bool FunctionID_is_nil(FunctionID id); * @param parent_task_id The task ID of the task that submitted this task. * @param parent_counter A counter indicating how many tasks were submitted by * the parent task prior to this one. - * @param actor_id The ID of the actor this task belongs to. - * @param actor_counter Number of tasks that have been executed on this actor. * @param function_id The function ID of the function to execute in this task. * @param num_args The number of arguments that this task has. * @param num_returns The number of return values that this task has. @@ -102,15 +92,14 @@ bool FunctionID_is_nil(FunctionID id); ignoring object ID arguments. * @return The partially constructed task_spec. */ -task_spec *start_construct_task_spec(UniqueID driver_id, - TaskID parent_task_id, - int64_t parent_counter, - UniqueID actor_id, - int64_t actor_counter, - FunctionID function_id, - int64_t num_args, - int64_t num_returns, - int64_t args_value_size); +void TaskSpec_start_construct(TaskBuilder *B, + UniqueID driver_id, + TaskID parent_task_id, + int64_t parent_counter, + UniqueID actor_id, + int64_t actor_counter, + FunctionID function_id, + int64_t num_returns); /** * Finish constructing a task_spec. This computes the task ID and the object IDs @@ -120,15 +109,7 @@ task_spec *start_construct_task_spec(UniqueID driver_id, * @param spec The task spec whose ID and return object IDs should be computed. * @return Void. */ -void finish_construct_task_spec(task_spec *spec); - -/** - * The size of the task in bytes. - * - * @param spec The task_spec in question. - * @return The size of the task_spec in bytes. - */ -int64_t task_spec_size(task_spec *spec); +uint8_t *TaskSpec_finish_construct(TaskBuilder *builder, int64_t *size); /** * Return the function ID of the task. @@ -136,7 +117,7 @@ int64_t task_spec_size(task_spec *spec); * @param spec The task_spec in question. * @return The function ID of the function to execute in this task. */ -FunctionID task_function(task_spec *spec); +FunctionID TaskSpec_function(TaskSpec *spec); /** * Return the actor ID of the task. @@ -144,7 +125,7 @@ FunctionID task_function(task_spec *spec); * @param spec The task_spec in question. * @return The actor ID of the actor the task is part of. */ -UniqueID task_spec_actor_id(task_spec *spec); +UniqueID TaskSpec_actor_id(TaskSpec *spec); /** * Return the actor counter of the task. This starts at 0 and increments by 1 @@ -153,7 +134,7 @@ UniqueID task_spec_actor_id(task_spec *spec); * @param spec The task_spec in question. * @return The actor counter of the task. */ -int64_t task_spec_actor_counter(task_spec *spec); +int64_t TaskSpec_actor_counter(TaskSpec *spec); /** * Return the driver ID of the task. @@ -161,7 +142,7 @@ int64_t task_spec_actor_counter(task_spec *spec); * @param spec The task_spec in question. * @return The driver ID of the task. */ -UniqueID task_spec_driver_id(task_spec *spec); +UniqueID TaskSpec_driver_id(TaskSpec *spec); /** * Return the task ID of the task. @@ -169,7 +150,7 @@ UniqueID task_spec_driver_id(task_spec *spec); * @param spec The task_spec in question. * @return The task ID of the task. */ -TaskID task_spec_id(task_spec *spec); +TaskID TaskSpec_task_id(TaskSpec *spec); /** * Get the number of arguments to this task. @@ -177,7 +158,7 @@ TaskID task_spec_id(task_spec *spec); * @param spec The task_spec in question. * @return The number of arguments to this task. */ -int64_t task_num_args(task_spec *spec); +int64_t TaskSpec_num_args(TaskSpec *spec); /** * Get the number of return values expected from this task. @@ -185,17 +166,16 @@ int64_t task_num_args(task_spec *spec); * @param spec The task_spec in question. * @return The number of return values expected from this task. */ -int64_t task_num_returns(task_spec *spec); +int64_t TaskSpec_num_returns(TaskSpec *spec); /** - * Get the type of an argument to this task. It should be either ARG_BY_REF or - * ARG_BY_VAL. + * Return true if this argument is passed by reference. * * @param spec The task_spec in question. * @param arg_index The index of the argument in question. - * @return The type of the argument. + * @return True if this argument is passed by reference. */ -int8_t task_arg_type(task_spec *spec, int64_t arg_index); +bool TaskSpec_arg_by_ref(TaskSpec *spec, int64_t arg_index); /** * Get a particular argument to this task. This assumes the argument is an @@ -205,7 +185,7 @@ int8_t task_arg_type(task_spec *spec, int64_t arg_index); * @param arg_index The index of the argument in question. * @return The argument at that index. */ -ObjectID task_arg_id(task_spec *spec, int64_t arg_index); +ObjectID TaskSpec_arg_id(TaskSpec *spec, int64_t arg_index); /** * Get a particular argument to this task. This assumes the argument is a value. @@ -214,7 +194,7 @@ ObjectID task_arg_id(task_spec *spec, int64_t arg_index); * @param arg_index The index of the argument in question. * @return The argument at that index. */ -uint8_t *task_arg_val(task_spec *spec, int64_t arg_index); +const uint8_t *TaskSpec_arg_val(TaskSpec *spec, int64_t arg_index); /** * Get the number of bytes in a particular argument to this task. This assumes @@ -224,7 +204,7 @@ uint8_t *task_arg_val(task_spec *spec, int64_t arg_index); * @param arg_index The index of the argument in question. * @return The number of bytes in the argument. */ -int64_t task_arg_length(task_spec *spec, int64_t arg_index); +int64_t TaskSpec_arg_length(TaskSpec *spec, int64_t arg_index); /** * Set the next task argument. Note that this API only allows you to set the @@ -235,7 +215,7 @@ int64_t task_arg_length(task_spec *spec, int64_t arg_index); * @return The number of task arguments that have been set before this one. This * is only used for testing. */ -int64_t task_args_add_ref(task_spec *spec, ObjectID obj_id); +void TaskSpec_args_add_ref(TaskBuilder *spec, ObjectID obj_id); /** * Set the next task argument. Note that this API only allows you to set the @@ -247,32 +227,9 @@ int64_t task_args_add_ref(task_spec *spec, ObjectID obj_id); * @return The number of task arguments that have been set before this one. This * is only used for testing. */ -int64_t task_args_add_val(task_spec *spec, uint8_t *data, int64_t length); - -/** - * Get a particular return object ID of a task. - * - * @param spec The task_spec in question. - * @param return_index The index of the return object ID in question. - * @return The relevant return object ID. - */ -ObjectID task_return(task_spec *spec, int64_t return_index); - -/** - * Indices into resource vectors. - * A resource vector maps a resource index to the number - * of units of that resource required. - * - * The total length of the resource vector is NUM_RESOURCE_INDICES. - */ -typedef enum { - /** Index for number of cpus the task requires. */ - CPU_RESOURCE_INDEX = 0, - /** Index for number of gpus the task requires. */ - GPU_RESOURCE_INDEX, - /** Total number of different resources in the system. */ - MAX_RESOURCE_INDEX -} resource_vector_index; +void TaskSpec_args_add_val(TaskBuilder *builder, + uint8_t *value, + int64_t length); /** * Set the value associated to a resource index. @@ -283,9 +240,18 @@ typedef enum { * this task needs or a value for an attribute this task requires. * @return Void. */ -void task_spec_set_required_resource(task_spec *spec, - int64_t resource_index, - double value); +void TaskSpec_set_required_resource(TaskBuilder *builder, + int64_t resource_index, + double value); + +/** + * Get a particular return object ID of a task. + * + * @param spec The task_spec in question. + * @param return_index The index of the return object ID in question. + * @return The relevant return object ID. + */ +ObjectID TaskSpec_return(TaskSpec *data, int64_t return_index); /** * Get the value associated to a resource index. @@ -294,8 +260,8 @@ void task_spec_set_required_resource(task_spec *spec, * @param resource_index Index of the resource. * @return How many of this resource the task needs to execute. */ -double task_spec_get_required_resource(const task_spec *spec, - int64_t resource_index); +double TaskSpec_get_required_resource(const TaskSpec *spec, + int64_t resource_index); /** * Compute the object id associated to a put call. @@ -307,21 +273,21 @@ double task_spec_get_required_resource(const task_spec *spec, ObjectID task_compute_put_id(TaskID task_id, int64_t put_index); /** - * Free a task_spec. + * Print the task as a humanly readable string. * - * @param The task_spec in question. + * @param spec The task_spec in question. + * @param output The buffer to write the string to. * @return Void. */ -void free_task_spec(task_spec *spec); +void TaskSpec_print(TaskSpec *spec, UT_string *output); /** - * Print the task as a humanly readable string. + * Free a task_spec. * - * @param spec The task_spec in question. - * @param output The buffer to write the string to. + * @param The task_spec in question. * @return Void. */ -void print_task(task_spec *spec, UT_string *output); +void TaskSpec_free(TaskSpec *spec); /** * ==== Task ==== @@ -353,7 +319,17 @@ typedef enum { /** A task is an execution of a task specification. It has a state of execution * (see scheduling_state) and the ID of the local scheduler it is scheduled on * or running on. */ -typedef struct TaskImpl Task; + +struct Task { + /** The scheduling state of the task. */ + int state; + /** The ID of the local scheduler involved. */ + DBClientID local_scheduler_id; + /** The size of the task specification for this task. */ + int64_t task_spec_size; + /** The task specification for this task. */ + TaskSpec spec; +}; /** * Allocate a new task. Must be freed with free_task after use. @@ -363,7 +339,10 @@ typedef struct TaskImpl Task; * @param local_scheduler_id The ID of the local scheduler that the task is * scheduled on, if any. */ -Task *Task_alloc(task_spec *spec, int state, DBClientID local_scheduler_id); +Task *Task_alloc(TaskSpec *spec, + int64_t task_spec_size, + int state, + DBClientID local_scheduler_id); /** * Create a copy of the task. Must be freed with free_task after use. @@ -383,13 +362,15 @@ int Task_state(Task *task); void Task_set_state(Task *task, int state); /** Local scheduler this task has been assigned to or is running on. */ -DBClientID Task_local_scheduler_id(Task *task); +DBClientID Task_local_scheduler(Task *task); /** Set the local scheduler ID for this task. */ -void Task_set_local_scheduler_id(Task *task, DBClientID local_scheduler_id); +void Task_set_local_scheduler(Task *task, DBClientID local_scheduler_id); /** Task specification of this task. */ -task_spec *Task_task_spec(Task *task); +TaskSpec *Task_task_spec(Task *task); + +int64_t Task_task_spec_size(Task *task); /** Task ID of this task. */ TaskID Task_task_id(Task *task); @@ -397,4 +378,4 @@ TaskID Task_task_id(Task *task); /** Free this task datastructure. */ void Task_free(Task *task); -#endif +#endif /* TASK_H */ diff --git a/src/common/test/db_tests.cc b/src/common/test/db_tests.cc index fb07b9f263c60..5d6b8308ab548 100644 --- a/src/common/test/db_tests.cc +++ b/src/common/test/db_tests.cc @@ -6,6 +6,7 @@ #include "event_loop.h" #include "test_common.h" +#include "example_task.h" #include "state/db.h" #include "state/object_table.h" #include "state/task_table.h" @@ -16,6 +17,8 @@ SUITE(db_tests); +TaskBuilder *g_task_builder = NULL; + /* Retry 10 times with an 100ms timeout. */ const int NUM_RETRIES = 10; const uint64_t TIMEOUT = 50; @@ -149,10 +152,11 @@ TEST task_table_test(void) { db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL); db_attach(db, loop, false); DBClientID local_scheduler_id = globally_unique_id(); - task_spec *spec = example_task_spec(1, 1); - task_table_test_task = - Task_alloc(spec, TASK_STATUS_SCHEDULED, local_scheduler_id); - free_task_spec(spec); + int64_t task_spec_size; + TaskSpec *spec = example_task_spec(1, 1, &task_spec_size); + task_table_test_task = Task_alloc(spec, task_spec_size, TASK_STATUS_SCHEDULED, + local_scheduler_id); + TaskSpec_free(spec); RetryInfo retry = { .num_retries = NUM_RETRIES, .timeout = TIMEOUT, @@ -183,10 +187,13 @@ TEST task_table_all_test(void) { DBHandle *db = db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL); db_attach(db, loop, false); - task_spec *spec = example_task_spec(1, 1); + int64_t task_spec_size; + TaskSpec *spec = example_task_spec(1, 1, &task_spec_size); /* Schedule two tasks on different local local schedulers. */ - Task *task1 = Task_alloc(spec, TASK_STATUS_SCHEDULED, globally_unique_id()); - Task *task2 = Task_alloc(spec, TASK_STATUS_SCHEDULED, globally_unique_id()); + Task *task1 = Task_alloc(spec, task_spec_size, TASK_STATUS_SCHEDULED, + globally_unique_id()); + Task *task2 = Task_alloc(spec, task_spec_size, TASK_STATUS_SCHEDULED, + globally_unique_id()); RetryInfo retry = { .num_retries = NUM_RETRIES, .timeout = TIMEOUT, .fail_callback = NULL, }; @@ -201,7 +208,7 @@ TEST task_table_all_test(void) { event_loop_add_timer(loop, 200, (event_loop_timer_handler) timeout_handler, NULL); event_loop_run(loop); - free_task_spec(spec); + TaskSpec_free(spec); db_disconnect(db); destroy_outstanding_callbacks(loop); event_loop_destroy(loop); @@ -237,6 +244,7 @@ SUITE(db_tests) { GREATEST_MAIN_DEFS(); int main(int argc, char **argv) { + g_task_builder = make_task_builder(); GREATEST_MAIN_BEGIN(); RUN_SUITE(db_tests); GREATEST_MAIN_END(); diff --git a/src/common/test/example_task.h b/src/common/test/example_task.h new file mode 100644 index 0000000000000..4c1eea6e68fce --- /dev/null +++ b/src/common/test/example_task.h @@ -0,0 +1,59 @@ +#ifndef EXAMPLE_TASK_H +#define EXAMPLE_TASK_H + +#include "task.h" + +extern TaskBuilder *g_task_builder; + +const int64_t arg_value_size = 1000; + +static inline TaskSpec *example_task_spec_with_args(int64_t num_args, + int64_t num_returns, + ObjectID arg_ids[], + int64_t *task_spec_size) { + TaskID parent_task_id = globally_unique_id(); + FunctionID func_id = globally_unique_id(); + TaskSpec_start_construct(g_task_builder, NIL_ID, parent_task_id, 0, + NIL_ACTOR_ID, 0, func_id, num_returns); + for (int64_t i = 0; i < num_args; ++i) { + ObjectID arg_id; + if (arg_ids == NULL) { + arg_id = globally_unique_id(); + } else { + arg_id = arg_ids[i]; + } + TaskSpec_args_add_ref(g_task_builder, arg_id); + } + return TaskSpec_finish_construct(g_task_builder, task_spec_size); +} + +static inline TaskSpec *example_task_spec(int64_t num_args, + int64_t num_returns, + int64_t *task_spec_size) { + return example_task_spec_with_args(num_args, num_returns, NULL, + task_spec_size); +} + +static inline Task *example_task_with_args(int64_t num_args, + int64_t num_returns, + int task_state, + ObjectID arg_ids[]) { + int64_t task_spec_size; + TaskSpec *spec = example_task_spec_with_args(num_args, num_returns, arg_ids, + &task_spec_size); + Task *instance = Task_alloc(spec, task_spec_size, task_state, NIL_ID); + TaskSpec_free(spec); + return instance; +} + +static inline Task *example_task(int64_t num_args, + int64_t num_returns, + int task_state) { + int64_t task_spec_size; + TaskSpec *spec = example_task_spec(num_args, num_returns, &task_spec_size); + Task *instance = Task_alloc(spec, task_spec_size, task_state, NIL_ID); + TaskSpec_free(spec); + return instance; +} + +#endif /* EXAMPLE_TASK_H */ diff --git a/src/common/test/object_table_tests.cc b/src/common/test/object_table_tests.cc index 24bd63eab26a9..6deb59097fdd7 100644 --- a/src/common/test/object_table_tests.cc +++ b/src/common/test/object_table_tests.cc @@ -1,6 +1,7 @@ #include "greatest.h" #include "event_loop.h" +#include "example_task.h" #include "test_common.h" #include "common.h" #include "state/object_table.h" @@ -11,6 +12,7 @@ SUITE(object_table_tests); static event_loop *g_loop; +TaskBuilder *g_task_builder = NULL; /* ==== Test adding and looking up metadata ==== */ @@ -18,7 +20,7 @@ int new_object_failed = 0; int new_object_succeeded = 0; ObjectID new_object_id; Task *new_object_task; -task_spec *new_object_task_spec; +TaskSpec *new_object_task_spec; TaskID new_object_task_id; void new_object_fail_callback(UniqueID id, @@ -68,7 +70,7 @@ TEST new_object_test(void) { new_object_id = globally_unique_id(); new_object_task = example_task(1, 1, TASK_STATUS_WAITING); new_object_task_spec = Task_task_spec(new_object_task); - new_object_task_id = task_spec_id(new_object_task_spec); + new_object_task_id = TaskSpec_task_id(new_object_task_spec); g_loop = event_loop_create(); DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL); @@ -902,6 +904,7 @@ SUITE(object_table_tests) { GREATEST_MAIN_DEFS(); int main(int argc, char **argv) { + g_task_builder = make_task_builder(); GREATEST_MAIN_BEGIN(); RUN_SUITE(object_table_tests); GREATEST_MAIN_END(); diff --git a/src/common/test/task_table_tests.cc b/src/common/test/task_table_tests.cc index bd7626d2a45e4..4047cb35c3a7c 100644 --- a/src/common/test/task_table_tests.cc +++ b/src/common/test/task_table_tests.cc @@ -1,6 +1,7 @@ #include "greatest.h" #include "event_loop.h" +#include "example_task.h" #include "test_common.h" #include "common.h" #include "state/object_table.h" @@ -12,6 +13,7 @@ SUITE(task_table_tests); event_loop *g_loop; +TaskBuilder *g_task_builder = NULL; /* ==== Test operations in non-failure scenario ==== */ @@ -418,6 +420,7 @@ SUITE(task_table_tests) { GREATEST_MAIN_DEFS(); int main(int argc, char **argv) { + g_task_builder = make_task_builder(); GREATEST_MAIN_BEGIN(); RUN_SUITE(task_table_tests); GREATEST_MAIN_END(); diff --git a/src/common/test/task_tests.cc b/src/common/test/task_tests.cc index 85085db36ae49..f348fe882e922 100644 --- a/src/common/test/task_tests.cc +++ b/src/common/test/task_tests.cc @@ -14,37 +14,39 @@ SUITE(task_tests); TEST task_test(void) { TaskID parent_task_id = globally_unique_id(); FunctionID func_id = globally_unique_id(); - task_spec *spec = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 4, 2, 10); - ASSERT(task_num_args(spec) == 4); - ASSERT(task_num_returns(spec) == 2); + TaskBuilder *builder = make_task_builder(); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 2); UniqueID arg1 = globally_unique_id(); - ASSERT(task_args_add_ref(spec, arg1) == 0); - ASSERT(task_args_add_val(spec, (uint8_t *) "hello", 5) == 1); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, (uint8_t *) "hello", 5); UniqueID arg2 = globally_unique_id(); - ASSERT(task_args_add_ref(spec, arg2) == 2); - ASSERT(task_args_add_val(spec, (uint8_t *) "world", 5) == 3); + TaskSpec_args_add_ref(builder, arg2); + TaskSpec_args_add_val(builder, (uint8_t *) "world", 5); /* Finish constructing the spec. This constructs the task ID and the * return IDs. */ - finish_construct_task_spec(spec); + int64_t size; + uint8_t *spec = TaskSpec_finish_construct(builder, &size); /* Check that the spec was constructed as expected. */ - ASSERT(task_num_args(spec) == 4); - ASSERT(task_num_returns(spec) == 2); - ASSERT(FunctionID_equal(task_function(spec), func_id)); - ASSERT(ObjectID_equal(task_arg_id(spec, 0), arg1)); - ASSERT(memcmp(task_arg_val(spec, 1), (uint8_t *) "hello", - task_arg_length(spec, 1)) == 0); - ASSERT(ObjectID_equal(task_arg_id(spec, 2), arg2)); - ASSERT(memcmp(task_arg_val(spec, 3), (uint8_t *) "world", - task_arg_length(spec, 3)) == 0); - - free_task_spec(spec); + ASSERT(TaskSpec_num_args(spec) == 4); + ASSERT(TaskSpec_num_returns(spec) == 2); + ASSERT(FunctionID_equal(TaskSpec_function(spec), func_id)); + ASSERT(ObjectID_equal(TaskSpec_arg_id(spec, 0), arg1)); + ASSERT(memcmp(TaskSpec_arg_val(spec, 1), (uint8_t *) "hello", + TaskSpec_arg_length(spec, 1)) == 0); + ASSERT(ObjectID_equal(TaskSpec_arg_id(spec, 2), arg2)); + ASSERT(memcmp(TaskSpec_arg_val(spec, 3), (uint8_t *) "world", + TaskSpec_arg_length(spec, 3)) == 0); + + TaskSpec_free(spec); + free_task_builder(builder); PASS(); } TEST deterministic_ids_test(void) { + TaskBuilder *builder = make_task_builder(); /* Define the inputs to the task construction. */ TaskID parent_task_id = globally_unique_id(); FunctionID func_id = globally_unique_id(); @@ -52,123 +54,132 @@ TEST deterministic_ids_test(void) { uint8_t *arg2 = (uint8_t *) "hello world"; /* Construct a first task. */ - task_spec *spec1 = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec1, arg1); - task_args_add_val(spec1, arg2, 11); - finish_construct_task_spec(spec1); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size1; + TaskSpec *spec1 = TaskSpec_finish_construct(builder, &size1); /* Construct a second identical task. */ - task_spec *spec2 = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec2, arg1); - task_args_add_val(spec2, arg2, 11); - finish_construct_task_spec(spec2); - - /* Check that these tasks have the same task IDs and the same return IDs.*/ - ASSERT(TaskID_equal(task_spec_id(spec1), task_spec_id(spec2))); - ASSERT(ObjectID_equal(task_return(spec1, 0), task_return(spec2, 0))); - ASSERT(ObjectID_equal(task_return(spec1, 1), task_return(spec2, 1))); - ASSERT(ObjectID_equal(task_return(spec1, 2), task_return(spec2, 2))); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size2; + TaskSpec *spec2 = TaskSpec_finish_construct(builder, &size2); + + /* Check that these tasks have the same task IDs and the same return IDs. */ + ASSERT(TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec2))); + ASSERT(ObjectID_equal(TaskSpec_return(spec1, 0), TaskSpec_return(spec2, 0))); + ASSERT(ObjectID_equal(TaskSpec_return(spec1, 1), TaskSpec_return(spec2, 1))); + ASSERT(ObjectID_equal(TaskSpec_return(spec1, 2), TaskSpec_return(spec2, 2))); /* Check that the return IDs are all distinct. */ - ASSERT(!ObjectID_equal(task_return(spec1, 0), task_return(spec2, 1))); - ASSERT(!ObjectID_equal(task_return(spec1, 0), task_return(spec2, 2))); - ASSERT(!ObjectID_equal(task_return(spec1, 1), task_return(spec2, 2))); + ASSERT(!ObjectID_equal(TaskSpec_return(spec1, 0), TaskSpec_return(spec2, 1))); + ASSERT(!ObjectID_equal(TaskSpec_return(spec1, 0), TaskSpec_return(spec2, 2))); + ASSERT(!ObjectID_equal(TaskSpec_return(spec1, 1), TaskSpec_return(spec2, 2))); /* Create more tasks that are only mildly different. */ /* Construct a task with a different parent task ID. */ - task_spec *spec3 = start_construct_task_spec( - NIL_ID, globally_unique_id(), 0, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec3, arg1); - task_args_add_val(spec3, arg2, 11); - finish_construct_task_spec(spec3); + TaskSpec_start_construct(builder, NIL_ID, globally_unique_id(), 0, + NIL_ACTOR_ID, 0, func_id, 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size3; + TaskSpec *spec3 = TaskSpec_finish_construct(builder, &size3); /* Construct a task with a different parent counter. */ - task_spec *spec4 = start_construct_task_spec( - NIL_ID, parent_task_id, 1, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec4, arg1); - task_args_add_val(spec4, arg2, 11); - finish_construct_task_spec(spec4); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 1, NIL_ACTOR_ID, 0, + func_id, 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size4; + TaskSpec *spec4 = TaskSpec_finish_construct(builder, &size4); /* Construct a task with a different function ID. */ - task_spec *spec5 = - start_construct_task_spec(NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, - globally_unique_id(), 2, 3, 11); - task_args_add_ref(spec5, arg1); - task_args_add_val(spec5, arg2, 11); - finish_construct_task_spec(spec5); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + globally_unique_id(), 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size5; + TaskSpec *spec5 = TaskSpec_finish_construct(builder, &size5); /* Construct a task with a different object ID argument. */ - task_spec *spec6 = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec6, globally_unique_id()); - task_args_add_val(spec6, arg2, 11); - finish_construct_task_spec(spec6); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 3); + TaskSpec_args_add_ref(builder, globally_unique_id()); + TaskSpec_args_add_val(builder, arg2, 11); + int64_t size6; + TaskSpec *spec6 = TaskSpec_finish_construct(builder, &size6); /* Construct a task with a different value argument. */ - task_spec *spec7 = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 2, 3, 11); - task_args_add_ref(spec7, arg1); - task_args_add_val(spec7, (uint8_t *) "hello_world", 11); - finish_construct_task_spec(spec7); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 3); + TaskSpec_args_add_ref(builder, arg1); + TaskSpec_args_add_val(builder, (uint8_t *) "hello_world", 11); + int64_t size7; + TaskSpec *spec7 = TaskSpec_finish_construct(builder, &size7); /* Check that the task IDs are all distinct from the original. */ - ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec3))); - ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec4))); - ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec5))); - ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec6))); - ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec7))); + ASSERT(!TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec3))); + ASSERT(!TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec4))); + ASSERT(!TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec5))); + ASSERT(!TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec6))); + ASSERT(!TaskID_equal(TaskSpec_task_id(spec1), TaskSpec_task_id(spec7))); /* Check that the return object IDs are distinct from the originals. */ - task_spec *specs[6] = {spec1, spec3, spec4, spec5, spec6, spec7}; + TaskSpec *specs[6] = {spec1, spec3, spec4, spec5, spec6, spec7}; for (int task_index1 = 0; task_index1 < 6; ++task_index1) { for (int return_index1 = 0; return_index1 < 3; ++return_index1) { for (int task_index2 = 0; task_index2 < 6; ++task_index2) { for (int return_index2 = 0; return_index2 < 3; ++return_index2) { if (task_index1 != task_index2 && return_index1 != return_index2) { ASSERT(!ObjectID_equal( - task_return(specs[task_index1], return_index1), - task_return(specs[task_index2], return_index2))); + TaskSpec_return(specs[task_index1], return_index1), + TaskSpec_return(specs[task_index2], return_index2))); } } } } } - free_task_spec(spec1); - free_task_spec(spec2); - free_task_spec(spec3); - free_task_spec(spec4); - free_task_spec(spec5); - free_task_spec(spec6); - free_task_spec(spec7); + TaskSpec_free(spec1); + TaskSpec_free(spec2); + TaskSpec_free(spec3); + TaskSpec_free(spec4); + TaskSpec_free(spec5); + TaskSpec_free(spec6); + TaskSpec_free(spec7); + free_task_builder(builder); PASS(); } TEST send_task(void) { + TaskBuilder *builder = make_task_builder(); TaskID parent_task_id = globally_unique_id(); FunctionID func_id = globally_unique_id(); - task_spec *spec = start_construct_task_spec( - NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 4, 2, 10); - task_args_add_ref(spec, globally_unique_id()); - task_args_add_val(spec, (uint8_t *) "Hello", 5); - task_args_add_val(spec, (uint8_t *) "World", 5); - task_args_add_ref(spec, globally_unique_id()); - finish_construct_task_spec(spec); + TaskSpec_start_construct(builder, NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, + func_id, 2); + TaskSpec_args_add_ref(builder, globally_unique_id()); + TaskSpec_args_add_val(builder, (uint8_t *) "Hello", 5); + TaskSpec_args_add_val(builder, (uint8_t *) "World", 5); + TaskSpec_args_add_ref(builder, globally_unique_id()); + int64_t size; + TaskSpec *spec = TaskSpec_finish_construct(builder, &size); int fd[2]; socketpair(AF_UNIX, SOCK_STREAM, 0, fd); - write_message(fd[0], SUBMIT_TASK, task_spec_size(spec), (uint8_t *) spec); + write_message(fd[0], SUBMIT_TASK, size, (uint8_t *) spec); int64_t type; int64_t length; uint8_t *message; read_message(fd[1], &type, &length, &message); - task_spec *result = (task_spec *) message; + TaskSpec *result = (TaskSpec *) message; ASSERT(type == SUBMIT_TASK); - ASSERT(memcmp(spec, result, task_spec_size(spec)) == 0); - ASSERT(memcmp(spec, result, task_spec_size(result)) == 0); - free(spec); + ASSERT(memcmp(spec, result, size) == 0); + TaskSpec_free(spec); free(result); + free_task_builder(builder); PASS(); } diff --git a/src/common/test/test_common.h b/src/common/test/test_common.h index 218a0a576e0b2..d4b81036dc706 100644 --- a/src/common/test/test_common.h +++ b/src/common/test/test_common.h @@ -3,64 +3,16 @@ #include +#include "common.h" #include "io.h" #include "hiredis/hiredis.h" #include "utstring.h" -#include "task.h" - #ifndef _WIN32 /* This function is actually not declared in standard POSIX, so declare it. */ extern int usleep(useconds_t usec); #endif -const int64_t arg_value_size = 1000; - -static inline task_spec *example_task_spec_with_args(int64_t num_args, - int64_t num_returns, - ObjectID arg_ids[]) { - TaskID parent_task_id = globally_unique_id(); - FunctionID func_id = globally_unique_id(); - task_spec *task = - start_construct_task_spec(NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, - func_id, num_args, num_returns, arg_value_size); - for (int64_t i = 0; i < num_args; ++i) { - ObjectID arg_id; - if (arg_ids == NULL) { - arg_id = globally_unique_id(); - } else { - arg_id = arg_ids[i]; - } - task_args_add_ref(task, arg_id); - } - finish_construct_task_spec(task); - return task; -} - -static inline task_spec *example_task_spec(int64_t num_args, - int64_t num_returns) { - return example_task_spec_with_args(num_args, num_returns, NULL); -} - -static inline Task *example_task_with_args(int64_t num_args, - int64_t num_returns, - int Task_state, - ObjectID arg_ids[]) { - task_spec *spec = example_task_spec_with_args(num_args, num_returns, arg_ids); - Task *instance = Task_alloc(spec, Task_state, NIL_ID); - free_task_spec(spec); - return instance; -} - -static inline Task *example_task(int64_t num_args, - int64_t num_returns, - int Task_state) { - task_spec *spec = example_task_spec(num_args, num_returns); - Task *instance = Task_alloc(spec, Task_state, NIL_ID); - free_task_spec(spec); - return instance; -} - /* I/O helper methods to retry binding to sockets. */ static inline UT_string *bind_ipc_sock_retry(const char *socket_name_format, int *fd) { diff --git a/src/global_scheduler/global_scheduler.cc b/src/global_scheduler/global_scheduler.cc index 575bb06b2e46d..c11987bfff5be 100644 --- a/src/global_scheduler/global_scheduler.cc +++ b/src/global_scheduler/global_scheduler.cc @@ -34,11 +34,11 @@ void assign_task_to_local_scheduler(GlobalSchedulerState *state, Task *task, DBClientID local_scheduler_id) { char id_string[ID_STRING_SIZE]; - task_spec *spec = Task_task_spec(task); + TaskSpec *spec = Task_task_spec(task); LOG_DEBUG("assigning task to local_scheduler_id = %s", ObjectID_to_string(local_scheduler_id, id_string, ID_STRING_SIZE)); Task_set_state(task, TASK_STATUS_SCHEDULED); - Task_set_local_scheduler_id(task, local_scheduler_id); + Task_set_local_scheduler(task, local_scheduler_id); LOG_DEBUG("Issuing a task table update for task = %s", ObjectID_to_string(Task_task_id(task), id_string, ID_STRING_SIZE)); UNUSED(id_string); @@ -52,12 +52,12 @@ void assign_task_to_local_scheduler(GlobalSchedulerState *state, local_scheduler->num_tasks_sent += 1; local_scheduler->num_recent_tasks_sent += 1; /* Resource accounting update for this local scheduler. */ - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { + for (int i = 0; i < ResourceIndex_MAX; i++) { /* Subtract task's resource from the cached dynamic resource capacity for * this local scheduler. This will be overwritten on the next heartbeat. */ local_scheduler->info.dynamic_resources[i] = MAX(0, local_scheduler->info.dynamic_resources[i] - - task_spec_get_required_resource(spec, i)); + TaskSpec_get_required_resource(spec, i)); } } diff --git a/src/global_scheduler/global_scheduler_algorithm.cc b/src/global_scheduler/global_scheduler_algorithm.cc index be31980d73cec..8c282a90ab48e 100644 --- a/src/global_scheduler/global_scheduler_algorithm.cc +++ b/src/global_scheduler/global_scheduler_algorithm.cc @@ -36,10 +36,10 @@ void GlobalSchedulerPolicyState_free(GlobalSchedulerPolicyState *policy_state) { * otherwise. */ bool constraints_satisfied_hard(const LocalScheduler *scheduler, - const task_spec *spec) { - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { + const TaskSpec *spec) { + for (int i = 0; i < ResourceIndex_MAX; i++) { if (scheduler->info.static_resources[i] < - task_spec_get_required_resource(spec, i)) { + TaskSpec_get_required_resource(spec, i)) { return false; } } @@ -56,7 +56,7 @@ void handle_task_round_robin(GlobalSchedulerState *state, CHECKM(utarray_len(state->local_schedulers) > 0, "No local schedulers. We currently don't handle this case."); LocalScheduler *scheduler = NULL; - task_spec *task_spec = Task_task_spec(task); + TaskSpec *task_spec = Task_task_spec(task); int i; int num_retries = 1; bool task_satisfied = false; @@ -83,22 +83,22 @@ void handle_task_round_robin(GlobalSchedulerState *state, } ObjectSizeEntry *create_object_size_hashmap(GlobalSchedulerState *state, - task_spec *task_spec, + TaskSpec *task_spec, bool *has_args_by_ref, int64_t *task_data_size) { ObjectSizeEntry *s = NULL, *object_size_table = NULL; *task_data_size = 0; - for (int i = 0; i < task_num_args(task_spec); i++) { + for (int i = 0; i < TaskSpec_num_args(task_spec); i++) { /* Object ids are only available for args by references. - * Args by value are serialized into the task_spec itself. + * Args by value are serialized into the TaskSpec itself. * We will only concern ourselves with args by ref for data size calculation */ - if (task_arg_type(task_spec, i) != ARG_BY_REF) { + if (!TaskSpec_arg_by_ref(task_spec, i)) { continue; } *has_args_by_ref = true; - ObjectID obj_id = task_arg_id(task_spec, i); + ObjectID obj_id = TaskSpec_arg_id(task_spec, i); /* Look up this object ID in the global scheduler object cache. */ SchedulerObjectInfo *obj_info_entry = NULL; HASH_FIND(hh, state->scheduler_object_info_table, &obj_id, sizeof(obj_id), @@ -245,14 +245,14 @@ double calculate_object_size_fraction(GlobalSchedulerState *state, double calculate_score_dynvec_normalized(GlobalSchedulerState *state, LocalScheduler *scheduler, - const task_spec *task_spec, + const TaskSpec *task_spec, double object_size_fraction) { /* The object size fraction is now calculated for this (task,node) pair. */ /* Construct the normalized dynamic resource attribute vector */ - double normalized_dynvec[MAX_RESOURCE_INDEX + 1]; + double normalized_dynvec[ResourceIndex_MAX + 1]; memset(&normalized_dynvec, 0, sizeof(normalized_dynvec)); - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { - double resreqval = task_spec_get_required_resource(task_spec, i); + for (int i = 0; i < ResourceIndex_MAX; i++) { + double resreqval = TaskSpec_get_required_resource(task_spec, i); if (resreqval <= 0) { /* Skip and leave normalized dynvec value == 0. */ continue; @@ -260,12 +260,12 @@ double calculate_score_dynvec_normalized(GlobalSchedulerState *state, normalized_dynvec[i] = MIN(1, scheduler->info.dynamic_resources[i] / resreqval); } - normalized_dynvec[MAX_RESOURCE_INDEX] = object_size_fraction; + normalized_dynvec[ResourceIndex_MAX] = object_size_fraction; /* Finally, calculate the score. */ double score = inner_product(normalized_dynvec, state->policy_state->resource_attribute_weight, - MAX_RESOURCE_INDEX + 1); + ResourceIndex_MAX + 1); return score; } @@ -278,7 +278,7 @@ double calculate_cost_pending(const GlobalSchedulerState *state, bool handle_task_waiting(GlobalSchedulerState *state, GlobalSchedulerPolicyState *policy_state, Task *task) { - task_spec *task_spec = Task_task_spec(task); + TaskSpec *task_spec = Task_task_spec(task); CHECKM(task_spec != NULL, "task wait handler encounted a task with NULL spec"); diff --git a/src/global_scheduler/global_scheduler_algorithm.h b/src/global_scheduler/global_scheduler_algorithm.h index 25358216823e9..d57b53402c626 100644 --- a/src/global_scheduler/global_scheduler_algorithm.h +++ b/src/global_scheduler/global_scheduler_algorithm.h @@ -23,7 +23,7 @@ typedef enum { struct GlobalSchedulerPolicyState { /** The index of the next local scheduler to assign a task to. */ int64_t round_robin_index; - double resource_attribute_weight[MAX_RESOURCE_INDEX + 1]; + double resource_attribute_weight[ResourceIndex_MAX + 1]; }; typedef struct { diff --git a/src/local_scheduler/CMakeLists.txt b/src/local_scheduler/CMakeLists.txt index 578b4627db3d2..c5c7899a03ce9 100644 --- a/src/local_scheduler/CMakeLists.txt +++ b/src/local_scheduler/CMakeLists.txt @@ -35,7 +35,7 @@ endif(APPLE) add_library(local_scheduler_client STATIC local_scheduler_client.cc) -target_link_libraries(local_scheduler_library local_scheduler_client ${COMMON_LIB} ${PYTHON_LIBRARIES}) +target_link_libraries(local_scheduler_library local_scheduler_client common ${PYTHON_LIBRARIES}) add_executable(local_scheduler local_scheduler.cc local_scheduler_algorithm.cc) target_link_libraries(local_scheduler local_scheduler_client common ${HIREDIS_LIB} plasma_lib) diff --git a/src/local_scheduler/local_scheduler.cc b/src/local_scheduler/local_scheduler.cc index 1bab3082eef4e..bdbcb31172bdb 100644 --- a/src/local_scheduler/local_scheduler.cc +++ b/src/local_scheduler/local_scheduler.cc @@ -37,20 +37,20 @@ UT_icd byte_icd = {sizeof(uint8_t), NULL, NULL, NULL}; * @return Void. */ void print_resource_info(const LocalSchedulerState *state, - const task_spec *spec) { + const TaskSpec *spec) { #if RAY_COMMON_LOG_LEVEL <= RAY_COMMON_DEBUG /* Print information about available and requested resources. */ char buftotal[256], bufavail[256], bufresreq[256]; snprintf(bufavail, sizeof(bufavail), "%8.4f %8.4f", - state->dynamic_resources[CPU_RESOURCE_INDEX], - state->dynamic_resources[GPU_RESOURCE_INDEX]); + state->dynamic_resources[ResourceIndex_CPU], + state->dynamic_resources[ResourceIndex_GPU]); snprintf(buftotal, sizeof(buftotal), "%8.4f %8.4f", - state->static_resources[CPU_RESOURCE_INDEX], - state->static_resources[GPU_RESOURCE_INDEX]); + state->static_resources[ResourceIndex_CPU], + state->static_resources[ResourceIndex_GPU]); if (spec) { snprintf(bufresreq, sizeof(bufresreq), "%8.4f %8.4f", - task_spec_get_required_resource(spec, CPU_RESOURCE_INDEX), - task_spec_get_required_resource(spec, GPU_RESOURCE_INDEX)); + task_spec_get_required_resource(spec, ResourceIndex_CPU), + task_spec_get_required_resource(spec, ResourceIndex_GPU)); } LOG_DEBUG("Resources: [total=%s][available=%s][requested=%s]", buftotal, bufavail, spec ? bufresreq : "n/a"); @@ -124,7 +124,7 @@ void kill_worker(LocalSchedulerClient *worker, bool cleanup) { /* Clean up the task in progress. */ if (worker->task_in_progress) { /* Return the resources that the worker was using. */ - task_spec *spec = Task_task_spec(worker->task_in_progress); + TaskSpec *spec = Task_task_spec(worker->task_in_progress); update_dynamic_resources(state, spec, true); /* Update the task table to reflect that the task failed to complete. */ if (state->db != NULL) { @@ -388,7 +388,7 @@ LocalSchedulerState *LocalSchedulerState_init( utarray_new(state->input_buffer, &byte_icd); /* Initialize resource vectors. */ - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { + for (int i = 0; i < ResourceIndex_MAX; i++) { state->static_resources[i] = state->dynamic_resources[i] = static_resource_conf[i]; } @@ -405,10 +405,10 @@ LocalSchedulerState *LocalSchedulerState_init( } void update_dynamic_resources(LocalSchedulerState *state, - task_spec *spec, + TaskSpec *spec, bool return_resources) { - for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) { - double resource = task_spec_get_required_resource(spec, i); + for (int i = 0; i < ResourceIndex_MAX; ++i) { + double resource = TaskSpec_get_required_resource(spec, i); if (!return_resources) { /* If we are not returning resources, we are leasing them, so we want to * subtract the resource quantities from our accounting. */ @@ -428,9 +428,10 @@ void update_dynamic_resources(LocalSchedulerState *state, } void assign_task_to_worker(LocalSchedulerState *state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, LocalSchedulerClient *worker) { - if (write_message(worker->sock, EXECUTE_TASK, task_spec_size(spec), + if (write_message(worker->sock, EXECUTE_TASK, task_spec_size, (uint8_t *) spec) < 0) { if (errno == EPIPE || errno == EBADF) { /* TODO(rkn): If this happens, the task should be added back to the task @@ -447,7 +448,7 @@ void assign_task_to_worker(LocalSchedulerState *state, /* Resource accounting: * Update dynamic resource vector in the local scheduler state. */ update_dynamic_resources(state, spec, false); - Task *task = Task_alloc(spec, TASK_STATUS_RUNNING, + Task *task = Task_alloc(spec, task_spec_size, TASK_STATUS_RUNNING, state->db ? get_db_client_id(state->db) : NIL_ID); /* Record which task this worker is executing. This will be freed in * process_message when the worker sends a GET_TASK message to the local @@ -497,16 +498,17 @@ void reconstruct_task_update_callback(Task *task, void *user_context) { /* Otherwise, the test-and-set succeeded, so resubmit the task for execution * to ensure that reconstruction will happen. */ LocalSchedulerState *state = (LocalSchedulerState *) user_context; - task_spec *spec = Task_task_spec(task); + TaskSpec *spec = Task_task_spec(task); /* If the task is an actor task, then we currently do not reconstruct it. * TODO(rkn): Handle this better. */ - CHECK(ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)); + CHECK(ActorID_equal(TaskSpec_actor_id(spec), NIL_ACTOR_ID)); /* Resubmit the task. */ - handle_task_submitted(state, state->algorithm_state, spec); + handle_task_submitted(state, state->algorithm_state, spec, + Task_task_spec_size(task)); /* Recursively reconstruct the task's inputs, if necessary. */ - for (int64_t i = 0; i < task_num_args(spec); ++i) { - if (task_arg_type(spec, i) == ARG_BY_REF) { - ObjectID arg_id = task_arg_id(spec, i); + for (int64_t i = 0; i < TaskSpec_num_args(spec); ++i) { + if (TaskSpec_arg_by_ref(spec, i)) { + ObjectID arg_id = TaskSpec_arg_id(spec, i); reconstruct_object(state, arg_id); } } @@ -601,22 +603,22 @@ void process_message(event_loop *loop, switch (type) { case SUBMIT_TASK: { - task_spec *spec = (task_spec *) utarray_front(state->input_buffer); + TaskSpec *spec = (TaskSpec *) utarray_front(state->input_buffer); /* Update the result table, which holds mappings of object ID -> ID of the * task that created it. */ if (state->db != NULL) { - TaskID task_id = task_spec_id(spec); - for (int64_t i = 0; i < task_num_returns(spec); ++i) { - ObjectID return_id = task_return(spec, i); + TaskID task_id = TaskSpec_task_id(spec); + for (int64_t i = 0; i < TaskSpec_num_returns(spec); ++i) { + ObjectID return_id = TaskSpec_return(spec, i); result_table_add(state->db, return_id, task_id, NULL, NULL, NULL); } } /* Handle the task submission. */ - if (ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) { - handle_task_submitted(state, state->algorithm_state, spec); + if (ActorID_equal(TaskSpec_actor_id(spec), NIL_ACTOR_ID)) { + handle_task_submitted(state, state->algorithm_state, spec, length); } else { - handle_actor_task_submitted(state, state->algorithm_state, spec); + handle_actor_task_submitted(state, state->algorithm_state, spec, length); } } break; @@ -693,7 +695,7 @@ void process_message(event_loop *loop, case GET_TASK: { /* If this worker reports a completed task: account for resources. */ if (worker->task_in_progress != NULL) { - task_spec *spec = Task_task_spec(worker->task_in_progress); + TaskSpec *spec = Task_task_spec(worker->task_in_progress); /* Return dynamic resources back for the task in progress. */ update_dynamic_resources(state, spec, true); /* If we're connected to Redis, update tables. */ @@ -798,14 +800,16 @@ void signal_handler(int signal) { /* End of the cleanup code. */ void handle_task_scheduled_callback(Task *original_task, void *user_context) { - task_spec *spec = Task_task_spec(original_task); - if (ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) { + TaskSpec *spec = Task_task_spec(original_task); + if (ActorID_equal(TaskSpec_actor_id(spec), NIL_ACTOR_ID)) { /* This task does not involve an actor. Handle it normally. */ - handle_task_scheduled(g_state, g_state->algorithm_state, spec); + handle_task_scheduled(g_state, g_state->algorithm_state, spec, + Task_task_spec_size(original_task)); } else { /* This task involves an actor. Call the scheduling algorithm's actor * handler. */ - handle_actor_task_scheduled(g_state, g_state->algorithm_state, spec); + handle_actor_task_scheduled(g_state, g_state->algorithm_state, spec, + Task_task_spec_size(original_task)); } } @@ -932,7 +936,7 @@ int main(int argc, char *argv[]) { char *node_ip_address = NULL; /* Comma-separated list of configured resource capabilities for this node. */ char *static_resource_list = NULL; - double static_resource_conf[MAX_RESOURCE_INDEX]; + double static_resource_conf[ResourceIndex_MAX]; /* The command to run when starting new workers. */ char *start_worker_command = NULL; /* The number of workers to start. */ @@ -978,15 +982,15 @@ int main(int argc, char *argv[]) { if (!static_resource_list) { /* Use defaults for this node's static resource configuration. */ memset(&static_resource_conf[0], 0, sizeof(static_resource_conf)); - static_resource_conf[CPU_RESOURCE_INDEX] = DEFAULT_NUM_CPUS; - static_resource_conf[GPU_RESOURCE_INDEX] = DEFAULT_NUM_GPUS; + static_resource_conf[ResourceIndex_CPU] = DEFAULT_NUM_CPUS; + static_resource_conf[ResourceIndex_GPU] = DEFAULT_NUM_GPUS; } else { /* Tokenize the string. */ const char delim[2] = ","; char *token; int idx = 0; /* Index into the resource vector. */ token = strtok(static_resource_list, delim); - while (token != NULL && idx < MAX_RESOURCE_INDEX) { + while (token != NULL && idx < ResourceIndex_MAX) { static_resource_conf[idx++] = atoi(token); /* Attempt to get the next token. */ token = strtok(NULL, delim); diff --git a/src/local_scheduler/local_scheduler.h b/src/local_scheduler/local_scheduler.h index 443cdb29796e0..9918f3ea95bde 100644 --- a/src/local_scheduler/local_scheduler.h +++ b/src/local_scheduler/local_scheduler.h @@ -39,7 +39,8 @@ void new_client_connection(event_loop *loop, * @return Void. */ void assign_task_to_worker(LocalSchedulerState *state, - task_spec *task, + TaskSpec *task, + int64_t task_spec_size, LocalSchedulerClient *worker); /** @@ -69,7 +70,7 @@ void process_plasma_notification(event_loop *loop, */ void reconstruct_object(LocalSchedulerState *state, ObjectID object_id); -void print_resource_info(const LocalSchedulerState *s, const task_spec *spec); +void print_resource_info(const LocalSchedulerState *s, const TaskSpec *spec); /** * Kill a worker. @@ -106,7 +107,7 @@ void start_worker(LocalSchedulerState *state, ActorID actor_id); * @return Void. */ void update_dynamic_resources(LocalSchedulerState *state, - task_spec *spec, + TaskSpec *spec, bool return_resources); /** The following methods are for testing purposes only. */ diff --git a/src/local_scheduler/local_scheduler_algorithm.cc b/src/local_scheduler/local_scheduler_algorithm.cc index eef1ca941ba6c..c6e10007d56b0 100644 --- a/src/local_scheduler/local_scheduler_algorithm.cc +++ b/src/local_scheduler/local_scheduler_algorithm.cc @@ -16,7 +16,8 @@ void remove_actor(SchedulingAlgorithmState *algorithm_state, ActorID actor_id); typedef struct task_queue_entry { /** The task that is queued. */ - task_spec *spec; + TaskSpec *spec; + int64_t task_spec_size; struct task_queue_entry *prev; struct task_queue_entry *next; } task_queue_entry; @@ -39,7 +40,9 @@ UT_icd task_queue_entry_icd = {sizeof(task_queue_entry *), NULL, NULL, NULL}; /** This is used to define the queue of actor task specs for which the * corresponding local scheduler is unknown. */ -UT_icd task_spec_icd = {sizeof(task_spec *), NULL, NULL, NULL}; +UT_icd task_spec_icd = {sizeof(TaskSpec *), NULL, NULL, NULL}; +/** This is used to keep track of task spec sizes in the above queue. */ +UT_icd task_spec_size_icd = {sizeof(int64_t), NULL, NULL, NULL}; /** This is used to define the queue of available workers. */ UT_icd worker_icd = {sizeof(LocalSchedulerClient *), NULL, NULL, NULL}; @@ -83,6 +86,8 @@ struct SchedulingAlgorithmState { * about a new local scheduler arrives, we will resubmit all of these tasks * locally. */ UT_array *cached_submitted_actor_tasks; + /** An array of task sizes of cached_submitted_actor_tasks. */ + UT_array *cached_submitted_actor_task_sizes; /** An array of pointers to workers in the worker pool. These are workers * that have registered a PID with us and that are now waiting to be * assigned a task to execute. */ @@ -119,6 +124,9 @@ SchedulingAlgorithmState *SchedulingAlgorithmState_init(void) { algorithm_state->dispatch_task_queue = NULL; utarray_new(algorithm_state->cached_submitted_actor_tasks, &task_spec_icd); + utarray_new(algorithm_state->cached_submitted_actor_task_sizes, + &task_spec_size_icd); + algorithm_state->local_actor_infos = NULL; utarray_new(algorithm_state->available_workers, &worker_icd); @@ -132,13 +140,13 @@ void SchedulingAlgorithmState_free(SchedulingAlgorithmState *algorithm_state) { task_queue_entry *elt, *tmp1; DL_FOREACH_SAFE(algorithm_state->waiting_task_queue, elt, tmp1) { DL_DELETE(algorithm_state->waiting_task_queue, elt); - free_task_spec(elt->spec); + free(elt->spec); free(elt); } /* Free all the tasks in the dispatch queue. */ DL_FOREACH_SAFE(algorithm_state->dispatch_task_queue, elt, tmp1) { DL_DELETE(algorithm_state->dispatch_task_queue, elt); - free_task_spec(elt->spec); + free(elt->spec); free(elt); } /* Remove all of the remaining actors. */ @@ -152,11 +160,12 @@ void SchedulingAlgorithmState_free(SchedulingAlgorithmState *algorithm_state) { /* Free the list of cached actor task specs and the task specs themselves. */ for (int i = 0; i < utarray_len(algorithm_state->cached_submitted_actor_tasks); ++i) { - task_spec **spec = (task_spec **) utarray_eltptr( + TaskSpec **spec = (TaskSpec **) utarray_eltptr( algorithm_state->cached_submitted_actor_tasks, i); free(*spec); } utarray_free(algorithm_state->cached_submitted_actor_tasks); + utarray_free(algorithm_state->cached_submitted_actor_task_sizes); /* Free the list of available workers. */ utarray_free(algorithm_state->available_workers); utarray_free(algorithm_state->executing_workers); @@ -195,7 +204,7 @@ void provide_scheduler_info(LocalSchedulerState *state, waiting_task_queue_length + dispatch_task_queue_length; info->available_workers = utarray_len(algorithm_state->available_workers); /* Copy static and dynamic resource information. */ - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { + for (int i = 0; i < ResourceIndex_MAX; i++) { info->dynamic_resources[i] = state->dynamic_resources[i]; info->static_resources[i] = state->static_resources[i]; } @@ -259,7 +268,7 @@ void remove_actor(SchedulingAlgorithmState *algorithm_state, ActorID actor_id) { task_queue_entry *task_queue_elt, *tmp; DL_FOREACH_SAFE(entry->task_queue, task_queue_elt, tmp) { DL_DELETE(entry->task_queue, task_queue_elt); - free_task_spec(task_queue_elt->spec); + free(task_queue_elt->spec); free(task_queue_elt); } /* Remove the entry from the hash table and free it. */ @@ -310,9 +319,10 @@ void handle_actor_worker_disconnect(LocalSchedulerState *state, */ void add_task_to_actor_queue(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, bool from_global_scheduler) { - ActorID actor_id = task_spec_actor_id(spec); + ActorID actor_id = TaskSpec_actor_id(spec); char tmp[ID_STRING_SIZE]; ObjectID_to_string(actor_id, tmp, ID_STRING_SIZE); DCHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID)); @@ -332,7 +342,7 @@ void add_task_to_actor_queue(LocalSchedulerState *state, CHECK(entry != NULL); } - int64_t task_counter = task_spec_actor_counter(spec); + int64_t task_counter = TaskSpec_actor_counter(spec); /* As a sanity check, the counter of the new task should be greater than the * number of tasks that have executed on this actor so far (since we are * guaranteeing in-order execution of the tasks on the actor). TODO(rkn): This @@ -342,8 +352,9 @@ void add_task_to_actor_queue(LocalSchedulerState *state, /* Create a new task queue entry. */ task_queue_entry *elt = (task_queue_entry *) malloc(sizeof(task_queue_entry)); - elt->spec = (task_spec *) malloc(task_spec_size(spec)); - memcpy(elt->spec, spec, task_spec_size(spec)); + elt->spec = (TaskSpec *) malloc(task_spec_size); + memcpy(elt->spec, spec, task_spec_size); + elt->task_spec_size = task_spec_size; /* Add the task spec to the actor's task queue in a manner that preserves the * order of the actor task counters. Iterate from the beginning of the queue * to find the right place to insert the task queue entry. TODO(pcm): This @@ -351,15 +362,15 @@ void add_task_to_actor_queue(LocalSchedulerState *state, * be optimized. */ task_queue_entry *current_entry = entry->task_queue; while (current_entry != NULL && current_entry->next != NULL && - task_counter > task_spec_actor_counter(current_entry->spec)) { + task_counter > TaskSpec_actor_counter(current_entry->spec)) { current_entry = current_entry->next; } DL_APPEND_ELEM(entry->task_queue, current_entry, elt); /* Update the task table. */ if (state->db != NULL) { - Task *task = - Task_alloc(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db)); + Task *task = Task_alloc(spec, task_spec_size, TASK_STATUS_QUEUED, + get_db_client_id(state->db)); if (from_global_scheduler) { /* If the task is from the global scheduler, it's already been added to * the task table, so just update the entry. */ @@ -403,7 +414,7 @@ bool dispatch_actor_task(LocalSchedulerState *state, * the actor. */ return false; } - int64_t next_task_counter = task_spec_actor_counter(entry->task_queue->spec); + int64_t next_task_counter = TaskSpec_actor_counter(entry->task_queue->spec); if (next_task_counter != entry->task_counter) { /* We cannot execute the next task on this actor without violating the * in-order execution guarantee for actor tasks. */ @@ -418,12 +429,13 @@ bool dispatch_actor_task(LocalSchedulerState *state, * as unavailable. */ task_queue_entry *first_task = entry->task_queue; entry->task_counter += 1; - assign_task_to_worker(state, first_task->spec, entry->worker); + assign_task_to_worker(state, first_task->spec, first_task->task_spec_size, + entry->worker); entry->worker_available = false; /* Remove the task from the actor's task queue. */ DL_DELETE(entry->task_queue, first_task); /* Free the task spec and the task queue entry. */ - free_task_spec(first_task->spec); + free(first_task->spec); free(first_task); return true; } @@ -479,12 +491,12 @@ void fetch_missing_dependency(LocalSchedulerState *state, void fetch_missing_dependencies(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, task_queue_entry *task_entry) { - task_spec *task = task_entry->spec; - int64_t num_args = task_num_args(task); + TaskSpec *task = task_entry->spec; + int64_t num_args = TaskSpec_num_args(task); int num_missing_dependencies = 0; for (int i = 0; i < num_args; ++i) { - if (task_arg_type(task, i) == ARG_BY_REF) { - ObjectID obj_id = task_arg_id(task, i); + if (TaskSpec_arg_by_ref(task, i)) { + ObjectID obj_id = TaskSpec_arg_id(task, i); object_entry *entry; HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id), entry); @@ -508,11 +520,11 @@ void fetch_missing_dependencies(LocalSchedulerState *state, * task are present in the local object store, otherwise it returns * false. */ -bool can_run(SchedulingAlgorithmState *algorithm_state, task_spec *task) { - int64_t num_args = task_num_args(task); +bool can_run(SchedulingAlgorithmState *algorithm_state, TaskSpec *task) { + int64_t num_args = TaskSpec_num_args(task); for (int i = 0; i < num_args; ++i) { - if (task_arg_type(task, i) == ARG_BY_REF) { - ObjectID obj_id = task_arg_id(task, i); + if (TaskSpec_arg_by_ref(task, i)) { + ObjectID obj_id = TaskSpec_arg_id(task, i); object_entry *entry; HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id), entry); @@ -580,7 +592,7 @@ void dispatch_tasks(LocalSchedulerState *state, } /* Terminate early if there are no more resources available. */ bool resources_available = false; - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { + for (int i = 0; i < ResourceIndex_MAX; i++) { if (state->dynamic_resources[i] > 0) { /* There are still resources left, continue checking tasks. */ resources_available = true; @@ -593,8 +605,8 @@ void dispatch_tasks(LocalSchedulerState *state, } /* Skip to the next task if this task cannot currently be satisfied. */ bool task_satisfied = true; - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { - if (task_spec_get_required_resource(elt->spec, i) > + for (int i = 0; i < ResourceIndex_MAX; i++) { + if (TaskSpec_get_required_resource(elt->spec, i) > state->dynamic_resources[i]) { /* Insufficient capacity for this task, proceed to the next task. */ task_satisfied = false; @@ -612,7 +624,7 @@ void dispatch_tasks(LocalSchedulerState *state, LocalSchedulerClient **worker = (LocalSchedulerClient **) utarray_back( algorithm_state->available_workers); /* Tell the available worker to execute the task. */ - assign_task_to_worker(state, elt->spec, *worker); + assign_task_to_worker(state, elt->spec, elt->task_spec_size, *worker); /* Remove the worker from the available queue, and add it to the executing * workers. */ utarray_pop_back(algorithm_state->available_workers); @@ -620,7 +632,7 @@ void dispatch_tasks(LocalSchedulerState *state, /* Dequeue the task and free the struct. */ print_resource_info(state, elt->spec); DL_DELETE(algorithm_state->dispatch_task_queue, elt); - free_task_spec(elt->spec); + free(elt->spec); free(elt); } /* End for each task in the dispatch queue. */ } @@ -641,20 +653,22 @@ void dispatch_tasks(LocalSchedulerState *state, */ task_queue_entry *queue_task(LocalSchedulerState *state, task_queue_entry **task_queue, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, bool from_global_scheduler) { /* Copy the spec and add it to the task queue. The allocated spec will be * freed when it is assigned to a worker. */ task_queue_entry *elt = (task_queue_entry *) malloc(sizeof(task_queue_entry)); - elt->spec = (task_spec *) malloc(task_spec_size(spec)); - memcpy(elt->spec, spec, task_spec_size(spec)); + elt->spec = (TaskSpec *) malloc(task_spec_size); + memcpy(elt->spec, spec, task_spec_size); + elt->task_spec_size = task_spec_size; DL_APPEND((*task_queue), elt); /* The task has been added to a local scheduler queue. Write the entry in the * task table to notify others that we have queued it. */ if (state->db != NULL) { - Task *task = - Task_alloc(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db)); + Task *task = Task_alloc(spec, task_spec_size, TASK_STATUS_QUEUED, + get_db_client_id(state->db)); if (from_global_scheduler) { /* If the task is from the global scheduler, it's already been added to * the task table, so just update the entry. */ @@ -684,11 +698,13 @@ task_queue_entry *queue_task(LocalSchedulerState *state, */ void queue_waiting_task(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, bool from_global_scheduler) { LOG_DEBUG("Queueing task in waiting queue"); - task_queue_entry *task_entry = queue_task( - state, &algorithm_state->waiting_task_queue, spec, from_global_scheduler); + task_queue_entry *task_entry = + queue_task(state, &algorithm_state->waiting_task_queue, spec, + task_spec_size, from_global_scheduler); /* If we're queueing this task in the waiting queue, there must be at least * one missing dependency, so record it. */ fetch_missing_dependencies(state, algorithm_state, task_entry); @@ -707,10 +723,11 @@ void queue_waiting_task(LocalSchedulerState *state, */ void queue_dispatch_task(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, bool from_global_scheduler) { LOG_DEBUG("Queueing task in dispatch queue"); - queue_task(state, &algorithm_state->dispatch_task_queue, spec, + queue_task(state, &algorithm_state->dispatch_task_queue, spec, task_spec_size, from_global_scheduler); } @@ -728,14 +745,17 @@ void queue_dispatch_task(LocalSchedulerState *state, */ void queue_task_locally(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, bool from_global_scheduler) { if (can_run(algorithm_state, spec)) { /* Dependencies are ready, so push the task to the dispatch queue. */ - queue_dispatch_task(state, algorithm_state, spec, from_global_scheduler); + queue_dispatch_task(state, algorithm_state, spec, task_spec_size, + from_global_scheduler); } else { /* Dependencies are not ready, so push the task to the waiting queue. */ - queue_waiting_task(state, algorithm_state, spec, from_global_scheduler); + queue_waiting_task(state, algorithm_state, spec, task_spec_size, + from_global_scheduler); } } @@ -751,7 +771,8 @@ void queue_task_locally(LocalSchedulerState *state, */ void give_task_to_local_scheduler(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec, + TaskSpec *spec, + int64_t task_spec_size, DBClientID local_scheduler_id) { if (DBClientID_equal(local_scheduler_id, get_db_client_id(state->db))) { LOG_WARN("Local scheduler is trying to assign a task to itself."); @@ -759,7 +780,8 @@ void give_task_to_local_scheduler(LocalSchedulerState *state, CHECK(state->db != NULL); /* Assign the task to the relevant local scheduler. */ DCHECK(state->config.global_scheduler_exists); - Task *task = Task_alloc(spec, TASK_STATUS_SCHEDULED, local_scheduler_id); + Task *task = Task_alloc(spec, task_spec_size, TASK_STATUS_SCHEDULED, + local_scheduler_id); task_table_add_task(state->db, task, NULL, NULL, NULL); } @@ -773,27 +795,27 @@ void give_task_to_local_scheduler(LocalSchedulerState *state, */ void give_task_to_global_scheduler(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec) { + TaskSpec *spec, + int64_t task_spec_size) { if (state->db == NULL || !state->config.global_scheduler_exists) { /* A global scheduler is not available, so queue the task locally. */ - queue_task_locally(state, algorithm_state, spec, false); + queue_task_locally(state, algorithm_state, spec, task_spec_size, false); return; } /* Pass on the task to the global scheduler. */ DCHECK(state->config.global_scheduler_exists); - Task *task = Task_alloc(spec, TASK_STATUS_WAITING, NIL_ID); + Task *task = Task_alloc(spec, task_spec_size, TASK_STATUS_WAITING, NIL_ID); DCHECK(state->db != NULL); task_table_add_task(state->db, task, NULL, NULL, NULL); } bool resource_constraints_satisfied(LocalSchedulerState *state, - task_spec *spec) { + TaskSpec *spec) { /* At the local scheduler, if required resource vector exceeds either static * or dynamic resource vector, the resource constraint is not satisfied. */ - for (int i = 0; i < MAX_RESOURCE_INDEX; i++) { - if (task_spec_get_required_resource(spec, i) > state->static_resources[i] || - task_spec_get_required_resource(spec, i) > - state->dynamic_resources[i]) { + for (int i = 0; i < ResourceIndex_MAX; i++) { + if (TaskSpec_get_required_resource(spec, i) > state->static_resources[i] || + TaskSpec_get_required_resource(spec, i) > state->dynamic_resources[i]) { return false; } } @@ -802,7 +824,8 @@ bool resource_constraints_satisfied(LocalSchedulerState *state, void handle_task_submitted(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec) { + TaskSpec *spec, + int64_t task_spec_size) { /* TODO(atumanov): if static is satisfied and local objects ready, but dynamic * resource is currently unavailable, then consider queueing task locally and * recheck dynamic next time. */ @@ -814,10 +837,10 @@ void handle_task_submitted(LocalSchedulerState *state, if (resource_constraints_satisfied(state, spec) && (utarray_len(algorithm_state->available_workers) > 0) && can_run(algorithm_state, spec)) { - queue_dispatch_task(state, algorithm_state, spec, false); + queue_dispatch_task(state, algorithm_state, spec, task_spec_size, false); } else { /* Give the task to the global scheduler to schedule, if it exists. */ - give_task_to_global_scheduler(state, algorithm_state, spec); + give_task_to_global_scheduler(state, algorithm_state, spec, task_spec_size); } /* Try to dispatch tasks, since we may have added one to the queue. */ @@ -826,8 +849,9 @@ void handle_task_submitted(LocalSchedulerState *state, void handle_actor_task_submitted(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec) { - ActorID actor_id = task_spec_actor_id(spec); + TaskSpec *spec, + int64_t task_spec_size) { + ActorID actor_id = TaskSpec_actor_id(spec); CHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID)); /* Find the local scheduler responsible for this actor. */ @@ -840,6 +864,8 @@ void handle_actor_task_submitted(LocalSchedulerState *state, * will be resubmitted (internally by the local scheduler) whenever a new * actor notification arrives. */ utarray_push_back(algorithm_state->cached_submitted_actor_tasks, &spec); + utarray_push_back(algorithm_state->cached_submitted_actor_task_sizes, + &task_spec_size); return; } @@ -847,13 +873,14 @@ void handle_actor_task_submitted(LocalSchedulerState *state, get_db_client_id(state->db))) { /* This local scheduler is responsible for the actor, so handle the task * locally. */ - add_task_to_actor_queue(state, algorithm_state, spec, false); + add_task_to_actor_queue(state, algorithm_state, spec, task_spec_size, + false); /* Attempt to dispatch tasks to this actor. */ dispatch_actor_task(state, algorithm_state, actor_id); } else { /* This local scheduler is not responsible for the task, so assign the task * directly to the actor that is responsible. */ - give_task_to_local_scheduler(state, algorithm_state, spec, + give_task_to_local_scheduler(state, algorithm_state, spec, task_spec_size, entry->local_scheduler_id); } } @@ -864,35 +891,43 @@ void handle_actor_creation_notification( ActorID actor_id) { int num_cached_actor_tasks = utarray_len(algorithm_state->cached_submitted_actor_tasks); + CHECK(num_cached_actor_tasks == + utarray_len(algorithm_state->cached_submitted_actor_task_sizes)); for (int i = 0; i < num_cached_actor_tasks; ++i) { - task_spec **spec = (task_spec **) utarray_eltptr( + TaskSpec **spec = (TaskSpec **) utarray_eltptr( algorithm_state->cached_submitted_actor_tasks, i); + int64_t *task_spec_size = (int64_t *) utarray_eltptr( + algorithm_state->cached_submitted_actor_task_sizes, i); /* Note that handle_actor_task_submitted may append the spec to the end of * the cached_submitted_actor_tasks array. */ - handle_actor_task_submitted(state, algorithm_state, *spec); + handle_actor_task_submitted(state, algorithm_state, *spec, *task_spec_size); } /* Remove all the tasks that were resubmitted. This does not erase the tasks * that were newly appended to the cached_submitted_actor_tasks array. */ utarray_erase(algorithm_state->cached_submitted_actor_tasks, 0, num_cached_actor_tasks); + utarray_erase(algorithm_state->cached_submitted_actor_task_sizes, 0, + num_cached_actor_tasks); } void handle_task_scheduled(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec) { + TaskSpec *spec, + int64_t task_spec_size) { /* This callback handles tasks that were assigned to this local scheduler by * the global scheduler, so we can safely assert that there is a connection to * the database. */ DCHECK(state->db != NULL); DCHECK(state->config.global_scheduler_exists); /* Push the task to the appropriate queue. */ - queue_task_locally(state, algorithm_state, spec, true); + queue_task_locally(state, algorithm_state, spec, task_spec_size, true); dispatch_tasks(state, algorithm_state); } void handle_actor_task_scheduled(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec) { + TaskSpec *spec, + int64_t task_spec_size) { /* This callback handles tasks that were assigned to this local scheduler by * the global scheduler or by other workers, so we can safely assert that * there is a connection to the database. */ @@ -900,7 +935,7 @@ void handle_actor_task_scheduled(LocalSchedulerState *state, DCHECK(state->config.global_scheduler_exists); /* Check that the task is meant to run on an actor that this local scheduler * is responsible for. */ - ActorID actor_id = task_spec_actor_id(spec); + ActorID actor_id = TaskSpec_actor_id(spec); DCHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID)); actor_map_entry *entry; HASH_FIND(hh, state->actor_mapping, &actor_id, sizeof(actor_id), entry); @@ -918,7 +953,7 @@ void handle_actor_task_scheduled(LocalSchedulerState *state, "corresponding actor_map_entry is not present. This should be rare."); } /* Push the task to the appropriate queue. */ - add_task_to_actor_queue(state, algorithm_state, spec, true); + add_task_to_actor_queue(state, algorithm_state, spec, task_spec_size, true); dispatch_actor_task(state, algorithm_state, actor_id); } @@ -1047,7 +1082,7 @@ void handle_worker_blocked(LocalSchedulerState *state, /* Return the resources that the blocked worker was using. */ CHECK(worker->task_in_progress != NULL); - task_spec *spec = Task_task_spec(worker->task_in_progress); + TaskSpec *spec = Task_task_spec(worker->task_in_progress); update_dynamic_resources(state, spec, true); /* Add the worker to the list of blocked workers. */ worker->is_blocked = true; @@ -1089,7 +1124,7 @@ void handle_worker_unblocked(LocalSchedulerState *state, * fixed by having blocked workers explicitly yield and wait to be given * back resources before continuing execution. */ CHECK(worker->task_in_progress != NULL); - task_spec *spec = Task_task_spec(worker->task_in_progress); + TaskSpec *spec = Task_task_spec(worker->task_in_progress); update_dynamic_resources(state, spec, false); /* Add the worker to the list of executing workers. */ worker->is_blocked = false; @@ -1171,11 +1206,11 @@ void handle_object_removed(LocalSchedulerState *state, task_queue_entry *elt, *tmp; /* Track the dependency for tasks that were in the waiting queue. */ DL_FOREACH(algorithm_state->waiting_task_queue, elt) { - task_spec *task = elt->spec; - int64_t num_args = task_num_args(task); + TaskSpec *task = elt->spec; + int64_t num_args = TaskSpec_num_args(task); for (int i = 0; i < num_args; ++i) { - if (task_arg_type(task, i) == ARG_BY_REF) { - ObjectID arg_id = task_arg_id(task, i); + if (TaskSpec_arg_by_ref(task, i)) { + ObjectID arg_id = TaskSpec_arg_id(task, i); if (ObjectID_equal(arg_id, removed_object_id)) { fetch_missing_dependency(state, algorithm_state, elt, removed_object_id); @@ -1186,11 +1221,11 @@ void handle_object_removed(LocalSchedulerState *state, /* Track the dependency for tasks that were in the dispatch queue. Remove * these tasks from the dispatch queue and push them to the waiting queue. */ DL_FOREACH_SAFE(algorithm_state->dispatch_task_queue, elt, tmp) { - task_spec *task = elt->spec; - int64_t num_args = task_num_args(task); + TaskSpec *task = elt->spec; + int64_t num_args = TaskSpec_num_args(task); for (int i = 0; i < num_args; ++i) { - if (task_arg_type(task, i) == ARG_BY_REF) { - ObjectID arg_id = task_arg_id(task, i); + if (TaskSpec_arg_by_ref(task, i)) { + ObjectID arg_id = TaskSpec_arg_id(task, i); if (ObjectID_equal(arg_id, removed_object_id)) { LOG_DEBUG("Moved task from dispatch queue back to waiting queue"); DL_DELETE(algorithm_state->dispatch_task_queue, elt); diff --git a/src/local_scheduler/local_scheduler_algorithm.h b/src/local_scheduler/local_scheduler_algorithm.h index de4225fe7d5fd..0c1025b9db71d 100644 --- a/src/local_scheduler/local_scheduler_algorithm.h +++ b/src/local_scheduler/local_scheduler_algorithm.h @@ -58,7 +58,8 @@ void provide_scheduler_info(LocalSchedulerState *state, */ void handle_task_submitted(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec); + TaskSpec *spec, + int64_t task_spec_size); /** * This version of handle_task_submitted is used when the task being submitted @@ -71,7 +72,8 @@ void handle_task_submitted(LocalSchedulerState *state, */ void handle_actor_task_submitted(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec); + TaskSpec *spec, + int64_t task_spec_size); /** * This function will be called when the local scheduler receives a notification @@ -99,7 +101,8 @@ void handle_actor_creation_notification( */ void handle_task_scheduled(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec); + TaskSpec *spec, + int64_t task_spec_size); /** * This function will be called when an actor task is assigned by the global @@ -113,7 +116,8 @@ void handle_task_scheduled(LocalSchedulerState *state, */ void handle_actor_task_scheduled(LocalSchedulerState *state, SchedulingAlgorithmState *algorithm_state, - task_spec *spec); + TaskSpec *spec, + int64_t task_spec_size); /** * This function is called if a new object becomes available in the local diff --git a/src/local_scheduler/local_scheduler_client.cc b/src/local_scheduler/local_scheduler_client.cc index 6488320ff58ec..8c3401e6dadeb 100644 --- a/src/local_scheduler/local_scheduler_client.cc +++ b/src/local_scheduler/local_scheduler_client.cc @@ -48,22 +48,22 @@ void local_scheduler_log_event(LocalSchedulerConnection *conn, free(message); } -void local_scheduler_submit(LocalSchedulerConnection *conn, task_spec *task) { - write_message(conn->conn, SUBMIT_TASK, task_spec_size(task), - (uint8_t *) task); +void local_scheduler_submit(LocalSchedulerConnection *conn, + TaskSpec *task, + int64_t task_size) { + write_message(conn->conn, SUBMIT_TASK, task_size, (uint8_t *) task); } -task_spec *local_scheduler_get_task(LocalSchedulerConnection *conn) { +TaskSpec *local_scheduler_get_task(LocalSchedulerConnection *conn, + int64_t *task_size) { write_message(conn->conn, GET_TASK, 0, NULL); int64_t type; - int64_t length; uint8_t *message; /* Receive a task from the local scheduler. This will block until the local * scheduler gives this client a task. */ - read_message(conn->conn, &type, &length, &message); + read_message(conn->conn, &type, task_size, &message); CHECK(type == EXECUTE_TASK); - task_spec *task = (task_spec *) message; - CHECK(length == task_spec_size(task)); + TaskSpec *task = (TaskSpec *) message; return task; } diff --git a/src/local_scheduler/local_scheduler_client.h b/src/local_scheduler/local_scheduler_client.h index 12dd7744a7fd3..4988bcf4a450e 100644 --- a/src/local_scheduler/local_scheduler_client.h +++ b/src/local_scheduler/local_scheduler_client.h @@ -39,7 +39,9 @@ void LocalSchedulerConnection_free(LocalSchedulerConnection *conn); * @param task The address of the task to submit. * @return Void. */ -void local_scheduler_submit(LocalSchedulerConnection *conn, task_spec *task); +void local_scheduler_submit(LocalSchedulerConnection *conn, + TaskSpec *task, + int64_t task_size); /** * Log an event to the event log. This will call RPUSH key value. We use RPUSH @@ -70,7 +72,8 @@ void local_scheduler_log_event(LocalSchedulerConnection *conn, * @param conn The connection information. * @return The address of the assigned task. */ -task_spec *local_scheduler_get_task(LocalSchedulerConnection *conn); +TaskSpec *local_scheduler_get_task(LocalSchedulerConnection *conn, + int64_t *task_size); /** * Tell the local scheduler that the client has finished executing a task. diff --git a/src/local_scheduler/local_scheduler_extension.cc b/src/local_scheduler/local_scheduler_extension.cc index 168d41d74f639..b0a08dde368d4 100644 --- a/src/local_scheduler/local_scheduler_extension.cc +++ b/src/local_scheduler/local_scheduler_extension.cc @@ -41,20 +41,21 @@ static PyObject *PyLocalSchedulerClient_submit(PyObject *self, PyObject *args) { } local_scheduler_submit( ((PyLocalSchedulerClient *) self)->local_scheduler_connection, - ((PyTask *) py_task)->spec); + ((PyTask *) py_task)->spec, ((PyTask *) py_task)->size); Py_RETURN_NONE; } // clang-format off static PyObject *PyLocalSchedulerClient_get_task(PyObject *self) { - task_spec *task_spec; + TaskSpec *task_spec; /* Drop the global interpreter lock while we get a task because * local_scheduler_get_task may block for a long time. */ + int64_t task_size; Py_BEGIN_ALLOW_THREADS task_spec = local_scheduler_get_task( - ((PyLocalSchedulerClient *) self)->local_scheduler_connection); + ((PyLocalSchedulerClient *) self)->local_scheduler_connection, &task_size); Py_END_ALLOW_THREADS - return PyTask_make(task_spec); + return PyTask_make(task_spec, task_size); } // clang-format on @@ -154,7 +155,7 @@ static PyMethodDef local_scheduler_methods[] = { "Return the object ID for a put call within a task."}, {"task_from_string", PyTask_from_string, METH_VARARGS, "Creates a Python PyTask object from a string representation of " - "task_spec."}, + "TaskSpec."}, {"task_to_string", PyTask_to_string, METH_VARARGS, "Translates a PyTask python object to a byte string."}, {NULL} /* Sentinel */ @@ -223,6 +224,8 @@ MOD_INIT(liblocal_scheduler_library) { PyModule_AddObject(m, "LocalSchedulerClient", (PyObject *) &PyLocalSchedulerClientType); + g_task_builder = make_task_builder(); + char local_scheduler_error[] = "local_scheduler.error"; LocalSchedulerError = PyErr_NewException(local_scheduler_error, NULL, NULL); Py_INCREF(LocalSchedulerError); diff --git a/src/local_scheduler/local_scheduler_shared.h b/src/local_scheduler/local_scheduler_shared.h index ba86efdb37e8c..f745acfe8b22d 100644 --- a/src/local_scheduler/local_scheduler_shared.h +++ b/src/local_scheduler/local_scheduler_shared.h @@ -94,10 +94,10 @@ typedef struct { UT_array *input_buffer; /** Vector of static attributes associated with the node owned by this local * scheduler. */ - double static_resources[MAX_RESOURCE_INDEX]; + double static_resources[ResourceIndex_MAX]; /** Vector of dynamic attributes associated with the node owned by this local * scheduler. */ - double dynamic_resources[MAX_RESOURCE_INDEX]; + double dynamic_resources[ResourceIndex_MAX]; } LocalSchedulerState; /** Contains all information associated with a local scheduler client. */ diff --git a/src/local_scheduler/test/local_scheduler_tests.cc b/src/local_scheduler/test/local_scheduler_tests.cc index 511125e279c60..527051d95da3d 100644 --- a/src/local_scheduler/test/local_scheduler_tests.cc +++ b/src/local_scheduler/test/local_scheduler_tests.cc @@ -9,6 +9,7 @@ #include "common.h" #include "test/test_common.h" +#include "test/example_task.h" #include "event_loop.h" #include "io.h" #include "utstring.h" @@ -23,6 +24,8 @@ SUITE(local_scheduler_tests); +TaskBuilder *g_task_builder = NULL; + const char *plasma_store_socket_name = "/tmp/plasma_store_socket_1"; const char *plasma_manager_socket_name_format = "/tmp/plasma_manager_socket_%d"; const char *local_scheduler_socket_name_format = @@ -56,8 +59,8 @@ LocalSchedulerMock *LocalSchedulerMock_init(int num_workers, const char *node_ip_address = "127.0.0.1"; const char *redis_addr = node_ip_address; int redis_port = 6379; - const double static_resource_conf[MAX_RESOURCE_INDEX] = {DEFAULT_NUM_CPUS, - DEFAULT_NUM_GPUS}; + const double static_resource_conf[ResourceIndex_MAX] = {DEFAULT_NUM_CPUS, + DEFAULT_NUM_GPUS}; LocalSchedulerMock *mock = (LocalSchedulerMock *) malloc(sizeof(LocalSchedulerMock)); memset(mock, 0, sizeof(LocalSchedulerMock)); @@ -155,8 +158,9 @@ TEST object_reconstruction_test(void) { LocalSchedulerConnection *worker = local_scheduler->conns[0]; /* Create a task with zero dependencies and one return value. */ - task_spec *spec = example_task_spec(0, 1); - ObjectID return_id = task_return(spec, 0); + int64_t task_size; + TaskSpec *spec = example_task_spec(0, 1, &task_size); + ObjectID return_id = TaskSpec_return(spec, 0); /* Add an empty object table entry for the object we want to reconstruct, to * simulate it having been created and evicted. */ @@ -176,15 +180,21 @@ TEST object_reconstruction_test(void) { if (pid == 0) { /* Make sure we receive the task twice. First from the initial submission, * and second from the reconstruct request. */ - local_scheduler_submit(worker, spec); - task_spec *task_assigned = local_scheduler_get_task(worker); - ASSERT_EQ(memcmp(task_assigned, spec, task_spec_size(spec)), 0); - task_spec *reconstruct_task = local_scheduler_get_task(worker); - ASSERT_EQ(memcmp(reconstruct_task, spec, task_spec_size(spec)), 0); + int64_t task_assigned_size; + local_scheduler_submit(worker, spec, task_size); + TaskSpec *task_assigned = + local_scheduler_get_task(worker, &task_assigned_size); + ASSERT_EQ(memcmp(task_assigned, spec, task_size), 0); + ASSERT_EQ(task_assigned_size, task_size); + int64_t reconstruct_task_size; + TaskSpec *reconstruct_task = + local_scheduler_get_task(worker, &reconstruct_task_size); + ASSERT_EQ(memcmp(reconstruct_task, spec, task_size), 0); + ASSERT_EQ(reconstruct_task_size, task_size); /* Clean up. */ - free_task_spec(reconstruct_task); - free_task_spec(task_assigned); - free_task_spec(spec); + free(reconstruct_task); + free(task_assigned); + TaskSpec_free(spec); LocalSchedulerMock_free(local_scheduler); exit(0); } else { @@ -196,12 +206,12 @@ TEST object_reconstruction_test(void) { /* Set the task's status to TASK_STATUS_DONE to prevent the race condition * that would suppress object reconstruction. */ Task *task = Task_alloc( - spec, TASK_STATUS_DONE, + spec, task_size, TASK_STATUS_DONE, get_db_client_id(local_scheduler->local_scheduler_state->db)); task_table_add_task(local_scheduler->local_scheduler_state->db, task, NULL, NULL, NULL); /* Trigger reconstruction, and run the event loop again. */ - ObjectID return_id = task_return(spec, 0); + ObjectID return_id = TaskSpec_return(spec, 0); local_scheduler_reconstruct_object(worker, return_id); event_loop_add_timer(local_scheduler->loop, 500, (event_loop_timer_handler) timeout_handler, NULL); @@ -209,7 +219,7 @@ TEST object_reconstruction_test(void) { /* Wait for the child process to exit and check that there are no tasks * left in the local scheduler's task queue. Then, clean up. */ wait(NULL); - free_task_spec(spec); + TaskSpec_free(spec); ASSERT_EQ(num_waiting_tasks( local_scheduler->local_scheduler_state->algorithm_state), 0); @@ -232,14 +242,15 @@ TEST object_reconstruction_recursive_test(void) { /* Create a chain of tasks, each one dependent on the one before it. Mark * each object as available so that tasks will run immediately. */ const int NUM_TASKS = 10; - task_spec *specs[NUM_TASKS]; - specs[0] = example_task_spec(0, 1); + TaskSpec *specs[NUM_TASKS]; + int64_t task_sizes[NUM_TASKS]; + specs[0] = example_task_spec(0, 1, &task_sizes[0]); for (int i = 1; i < NUM_TASKS; ++i) { - ObjectID arg_id = task_return(specs[i - 1], 0); + ObjectID arg_id = TaskSpec_return(specs[i - 1], 0); handle_object_available( local_scheduler->local_scheduler_state, local_scheduler->local_scheduler_state->algorithm_state, arg_id); - specs[i] = example_task_spec_with_args(1, 1, &arg_id); + specs[i] = example_task_spec_with_args(1, 1, &arg_id, &task_sizes[i]); } /* Add an empty object table entry for each object we want to reconstruct, to @@ -247,7 +258,7 @@ TEST object_reconstruction_recursive_test(void) { const char *client_id = "clientid"; redisContext *context = redisConnect("127.0.0.1", 6379); for (int i = 0; i < NUM_TASKS; ++i) { - ObjectID return_id = task_return(specs[i], 0); + ObjectID return_id = TaskSpec_return(specs[i], 0); redisReply *reply = (redisReply *) redisCommand( context, "RAY.OBJECT_TABLE_ADD %b %ld %b %s", return_id.id, sizeof(return_id.id), 1, NIL_DIGEST, (size_t) DIGEST_SIZE, client_id); @@ -263,32 +274,34 @@ TEST object_reconstruction_recursive_test(void) { if (pid == 0) { /* Submit the tasks, and make sure each one gets assigned to a worker. */ for (int i = 0; i < NUM_TASKS; ++i) { - local_scheduler_submit(worker, specs[i]); + local_scheduler_submit(worker, specs[i], task_sizes[i]); } /* Make sure we receive each task from the initial submission. */ for (int i = 0; i < NUM_TASKS; ++i) { - task_spec *task_assigned = local_scheduler_get_task(worker); - ASSERT_EQ(memcmp(task_assigned, specs[i], task_spec_size(task_assigned)), - 0); - free_task_spec(task_assigned); + int64_t task_size; + TaskSpec *task_assigned = local_scheduler_get_task(worker, &task_size); + ASSERT_EQ(memcmp(task_assigned, specs[i], task_sizes[i]), 0); + ASSERT_EQ(task_size, task_sizes[i]); + free(task_assigned); } /* Check that the workers receive all tasks in the final return object's * lineage during reconstruction. */ for (int i = 0; i < NUM_TASKS; ++i) { - task_spec *task_assigned = local_scheduler_get_task(worker); + int64_t task_assigned_size; + TaskSpec *task_assigned = + local_scheduler_get_task(worker, &task_assigned_size); bool found = false; for (int j = 0; j < NUM_TASKS; ++j) { if (specs[j] == NULL) { continue; } - if (memcmp(task_assigned, specs[j], task_spec_size(task_assigned)) == - 0) { + if (memcmp(task_assigned, specs[j], task_assigned_size) == 0) { found = true; - free_task_spec(specs[j]); + TaskSpec_free(specs[j]); specs[j] = NULL; } } - free_task_spec(task_assigned); + free(task_assigned); ASSERT(found); } LocalSchedulerMock_free(local_scheduler); @@ -302,13 +315,13 @@ TEST object_reconstruction_recursive_test(void) { /* Set the final task's status to TASK_STATUS_DONE to prevent the race * condition that would suppress object reconstruction. */ Task *last_task = Task_alloc( - specs[NUM_TASKS - 1], TASK_STATUS_DONE, + specs[NUM_TASKS - 1], task_sizes[NUM_TASKS - 1], TASK_STATUS_DONE, get_db_client_id(local_scheduler->local_scheduler_state->db)); task_table_add_task(local_scheduler->local_scheduler_state->db, last_task, NULL, NULL, NULL); /* Trigger reconstruction for the last object, and run the event loop * again. */ - ObjectID return_id = task_return(specs[NUM_TASKS - 1], 0); + ObjectID return_id = TaskSpec_return(specs[NUM_TASKS - 1], 0); local_scheduler_reconstruct_object(worker, return_id); event_loop_add_timer(local_scheduler->loop, 500, (event_loop_timer_handler) timeout_handler, NULL); @@ -323,7 +336,7 @@ TEST object_reconstruction_recursive_test(void) { local_scheduler->local_scheduler_state->algorithm_state), 0); for (int i = 0; i < NUM_TASKS; ++i) { - free_task_spec(specs[i]); + TaskSpec_free(specs[i]); } LocalSchedulerMock_free(local_scheduler); PASS(); @@ -334,35 +347,41 @@ TEST object_reconstruction_recursive_test(void) { * Test that object reconstruction gets suppressed when there is a location * listed for the object in the object table. */ -task_spec *object_reconstruction_suppression_spec; +TaskSpec *object_reconstruction_suppression_spec; +int64_t object_reconstruction_suppression_size; void object_reconstruction_suppression_callback(ObjectID object_id, void *user_context) { /* Submit the task after adding the object to the object table. */ LocalSchedulerConnection *worker = (LocalSchedulerConnection *) user_context; - local_scheduler_submit(worker, object_reconstruction_suppression_spec); + local_scheduler_submit(worker, object_reconstruction_suppression_spec, + object_reconstruction_suppression_size); } TEST object_reconstruction_suppression_test(void) { LocalSchedulerMock *local_scheduler = LocalSchedulerMock_init(0, 1); LocalSchedulerConnection *worker = local_scheduler->conns[0]; - object_reconstruction_suppression_spec = example_task_spec(0, 1); - ObjectID return_id = task_return(object_reconstruction_suppression_spec, 0); + object_reconstruction_suppression_spec = + example_task_spec(0, 1, &object_reconstruction_suppression_size); + ObjectID return_id = + TaskSpec_return(object_reconstruction_suppression_spec, 0); pid_t pid = fork(); if (pid == 0) { /* Make sure we receive the task once. This will block until the * object_table_add callback completes. */ - task_spec *task_assigned = local_scheduler_get_task(worker); + int64_t task_assigned_size; + TaskSpec *task_assigned = + local_scheduler_get_task(worker, &task_assigned_size); ASSERT_EQ(memcmp(task_assigned, object_reconstruction_suppression_spec, - task_spec_size(object_reconstruction_suppression_spec)), + object_reconstruction_suppression_size), 0); /* Trigger a reconstruction. We will check that no tasks get queued as a * result of this line in the event loop process. */ local_scheduler_reconstruct_object(worker, return_id); /* Clean up. */ - free_task_spec(task_assigned); - free_task_spec(object_reconstruction_suppression_spec); + free(task_assigned); + TaskSpec_free(object_reconstruction_suppression_spec); LocalSchedulerMock_free(local_scheduler); exit(0); } else { @@ -389,7 +408,7 @@ TEST object_reconstruction_suppression_test(void) { ASSERT_EQ(num_dispatch_tasks( local_scheduler->local_scheduler_state->algorithm_state), 0); - free_task_spec(object_reconstruction_suppression_spec); + TaskSpec_free(object_reconstruction_suppression_spec); db_disconnect(db); LocalSchedulerMock_free(local_scheduler); PASS(); @@ -403,12 +422,13 @@ TEST task_dependency_test(void) { /* Get the first worker. */ LocalSchedulerClient *worker = *((LocalSchedulerClient **) utarray_eltptr(state->workers, 0)); - task_spec *spec = example_task_spec(1, 1); - ObjectID oid = task_arg_id(spec, 0); + int64_t task_size; + TaskSpec *spec = example_task_spec(1, 1, &task_size); + ObjectID oid = TaskSpec_arg_id(spec, 0); /* Check that the task gets queued in the waiting queue if the task is * submitted, but the input and workers are not available. */ - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); ASSERT_EQ(num_waiting_tasks(algorithm_state), 1); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0); /* Once the input is available, the task gets moved to the dispatch queue. */ @@ -424,7 +444,7 @@ TEST task_dependency_test(void) { /* Check that the task gets queued in the waiting queue if the task is * submitted and a worker is available, but the input is not. */ handle_object_removed(state, oid); - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); handle_worker_available(state, algorithm_state, worker); ASSERT_EQ(num_waiting_tasks(algorithm_state), 1); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0); @@ -436,7 +456,7 @@ TEST task_dependency_test(void) { /* Check that the task gets queued in the dispatch queue if the task is * submitted and the input is available, but no worker is available yet. */ - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); ASSERT_EQ(num_waiting_tasks(algorithm_state), 0); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 1); /* Once a worker is available, the task gets assigned. */ @@ -448,7 +468,7 @@ TEST task_dependency_test(void) { /* If an object gets removed, check the first scenario again, where the task * gets queued in the waiting task if the task is submitted and a worker is * available, but the input is not. */ - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); ASSERT_EQ(num_waiting_tasks(algorithm_state), 0); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 1); /* If the input is removed while a task is in the dispatch queue, the task @@ -466,7 +486,7 @@ TEST task_dependency_test(void) { ASSERT_EQ(num_waiting_tasks(algorithm_state), 0); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0); - free_task_spec(spec); + TaskSpec_free(spec); LocalSchedulerMock_free(local_scheduler); PASS(); } @@ -478,13 +498,14 @@ TEST task_multi_dependency_test(void) { /* Get the first worker. */ LocalSchedulerClient *worker = *((LocalSchedulerClient **) utarray_eltptr(state->workers, 0)); - task_spec *spec = example_task_spec(2, 1); - ObjectID oid1 = task_arg_id(spec, 0); - ObjectID oid2 = task_arg_id(spec, 1); + int64_t task_size; + TaskSpec *spec = example_task_spec(2, 1, &task_size); + ObjectID oid1 = TaskSpec_arg_id(spec, 0); + ObjectID oid2 = TaskSpec_arg_id(spec, 1); /* Check that the task gets queued in the waiting queue if the task is * submitted, but the inputs and workers are not available. */ - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); ASSERT_EQ(num_waiting_tasks(algorithm_state), 1); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0); /* Check that the task stays in the waiting queue if only one input becomes @@ -504,7 +525,7 @@ TEST task_multi_dependency_test(void) { /* Check that the task gets queued in the dispatch queue if the task is * submitted and the inputs are available, but no worker is available yet. */ - handle_task_submitted(state, algorithm_state, spec); + handle_task_submitted(state, algorithm_state, spec, task_size); ASSERT_EQ(num_waiting_tasks(algorithm_state), 0); ASSERT_EQ(num_dispatch_tasks(algorithm_state), 1); /* If any input is removed while a task is in the dispatch queue, the task @@ -540,7 +561,7 @@ TEST task_multi_dependency_test(void) { ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0); reset_worker(local_scheduler, worker); - free_task_spec(spec); + TaskSpec_free(spec); LocalSchedulerMock_free(local_scheduler); PASS(); } @@ -633,6 +654,7 @@ SUITE(local_scheduler_tests) { GREATEST_MAIN_DEFS(); int main(int argc, char **argv) { + g_task_builder = make_task_builder(); GREATEST_MAIN_BEGIN(); RUN_SUITE(local_scheduler_tests); GREATEST_MAIN_END(); diff --git a/src/plasma/CMakeLists.txt b/src/plasma/CMakeLists.txt index b7ee8a9a28a07..86e489d5c8736 100644 --- a/src/plasma/CMakeLists.txt +++ b/src/plasma/CMakeLists.txt @@ -30,8 +30,7 @@ add_custom_command( VERBATIM ) -add_custom_target(protocol_fbs) -add_dependencies(protocol_fbs flatbuffers_ep) +add_dependencies(gen_plasma_fbs flatbuffers_ep) if(UNIX AND NOT APPLE) link_libraries(rt) @@ -58,9 +57,9 @@ if(APPLE) endif(APPLE) if(APPLE) - target_link_libraries(plasma -Wl,-force_load,${FLATBUFFERS_STATIC_LIB} ${COMMON_LIB} ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB}) + target_link_libraries(plasma -Wl,-force_load,${FLATBUFFERS_STATIC_LIB} common ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB}) else(APPLE) - target_link_libraries(plasma -Wl,--whole-archive ${FLATBUFFERS_STATIC_LIB} -Wl,--no-whole-archive ${COMMON_LIB} ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB}) + target_link_libraries(plasma -Wl,--whole-archive ${FLATBUFFERS_STATIC_LIB} -Wl,--no-whole-archive common ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB}) endif(APPLE) include_directories("${FLATBUFFERS_INCLUDE_DIR}") diff --git a/src/plasma/malloc.c b/src/plasma/malloc.c index 38a9734b4474c..a88d4d96944cf 100644 --- a/src/plasma/malloc.c +++ b/src/plasma/malloc.c @@ -7,7 +7,6 @@ #include #include "common.h" -#include "plasma.h" #include "uthash.h" void *fake_mmap(size_t); diff --git a/src/plasma/plasma_client.cc b/src/plasma/plasma_client.cc index 66ae58dc58af8..97ac1a2ed7a09 100644 --- a/src/plasma/plasma_client.cc +++ b/src/plasma/plasma_client.cc @@ -27,10 +27,9 @@ #include "plasma_client.h" #include "uthash.h" #include "utlist.h" -#include "sha256.h" extern "C" { - +#include "sha256.h" #include "fling.h" #define XXH_STATIC_LINKING_ONLY diff --git a/src/plasma/plasma_protocol.cc b/src/plasma/plasma_protocol.cc index cb3c8dc7efb4f..656c982e38810 100644 --- a/src/plasma/plasma_protocol.cc +++ b/src/plasma/plasma_protocol.cc @@ -2,46 +2,9 @@ #include "format/plasma_generated.h" #include "plasma_protocol.h" +#include "common_protocol.h" #include "io.h" -/** - * Convert an object ID to a flatbuffer string. - * - * @param fbb Reference to the flatbuffer builder. - * @param object_id The object ID to be converted. - * @return The flatbuffer string contining the object ID. - */ -flatbuffers::Offset to_flat( - flatbuffers::FlatBufferBuilder &fbb, - ObjectID object_id) { - return fbb.CreateString((char *) &object_id.id[0], sizeof(object_id.id)); -} - -/** - * Convert a flatbuffer string to an object ID. - * - * @param string The flatbuffer string. - * @return The object ID. - */ -ObjectID from_flat(const flatbuffers::String *string) { - ObjectID object_id; - CHECK(string->size() == sizeof(object_id.id)); - memcpy(&object_id.id[0], string->data(), sizeof(object_id.id)); - return object_id; -} - -flatbuffers::Offset< - flatbuffers::Vector>> -to_flat(flatbuffers::FlatBufferBuilder &fbb, - ObjectID object_ids[], - int64_t num_objects) { - std::vector> results; - for (size_t i = 0; i < num_objects; i++) { - results.push_back(to_flat(fbb, object_ids[i])); - } - return fbb.CreateVector(results); -} - #define FLATBUFFER_BUILDER_DEFAULT_SIZE 1024 protocol_builder *make_protocol_builder(void) { @@ -68,7 +31,7 @@ int plasma_send_CreateRequest(int sock, int64_t data_size, int64_t metadata_size) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaCreateRequest(fbb, to_flat(fbb, object_id), + auto message = CreatePlasmaCreateRequest(fbb, to_flatbuf(fbb, object_id), data_size, metadata_size); fbb.Finish(message); return write_message(sock, MessageType_PlasmaCreateRequest, fbb.GetSize(), @@ -83,7 +46,7 @@ void plasma_read_CreateRequest(uint8_t *data, auto message = flatbuffers::GetRoot(data); *data_size = message->data_size(); *metadata_size = message->metadata_size(); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); } int plasma_send_CreateReply(int sock, @@ -95,8 +58,9 @@ int plasma_send_CreateReply(int sock, PlasmaObjectSpec plasma_object( object->handle.store_fd, object->handle.mmap_size, object->data_offset, object->data_size, object->metadata_offset, object->metadata_size); - auto message = CreatePlasmaCreateReply( - fbb, to_flat(fbb, object_id), &plasma_object, (PlasmaError) error_code); + auto message = + CreatePlasmaCreateReply(fbb, to_flatbuf(fbb, object_id), &plasma_object, + (PlasmaError) error_code); fbb.Finish(message); return write_message(sock, MessageType_PlasmaCreateReply, fbb.GetSize(), fbb.GetBufferPointer()); @@ -108,7 +72,7 @@ void plasma_read_CreateReply(uint8_t *data, int *error_code) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); object->handle.store_fd = message->plasma_object()->segment_index(); object->handle.mmap_size = message->plasma_object()->mmap_size(); object->data_offset = message->plasma_object()->data_offset(); @@ -127,7 +91,7 @@ int plasma_send_SealRequest(int sock, flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto digest_string = fbb.CreateString((char *) digest, DIGEST_SIZE); auto message = - CreatePlasmaSealRequest(fbb, to_flat(fbb, object_id), digest_string); + CreatePlasmaSealRequest(fbb, to_flatbuf(fbb, object_id), digest_string); fbb.Finish(message); return write_message(sock, MessageType_PlasmaSealRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -138,7 +102,7 @@ void plasma_read_SealRequest(uint8_t *data, unsigned char *digest) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); CHECK(message->digest()->size() == DIGEST_SIZE); memcpy(digest, message->digest()->data(), DIGEST_SIZE); } @@ -148,8 +112,8 @@ int plasma_send_SealReply(int sock, ObjectID object_id, int error) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = - CreatePlasmaSealReply(fbb, to_flat(fbb, object_id), (PlasmaError) error); + auto message = CreatePlasmaSealReply(fbb, to_flatbuf(fbb, object_id), + (PlasmaError) error); fbb.Finish(message); return write_message(sock, MessageType_PlasmaSealReply, fbb.GetSize(), fbb.GetBufferPointer()); @@ -158,7 +122,7 @@ int plasma_send_SealReply(int sock, void plasma_read_SealReply(uint8_t *data, ObjectID *object_id, int *error) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *error = message->error(); } @@ -168,7 +132,7 @@ int plasma_send_ReleaseRequest(int sock, protocol_builder *B, ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaSealRequest(fbb, to_flat(fbb, object_id)); + auto message = CreatePlasmaSealRequest(fbb, to_flatbuf(fbb, object_id)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaReleaseRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -177,7 +141,7 @@ int plasma_send_ReleaseRequest(int sock, void plasma_read_ReleaseRequest(uint8_t *data, ObjectID *object_id) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); } int plasma_send_ReleaseReply(int sock, @@ -185,7 +149,7 @@ int plasma_send_ReleaseReply(int sock, ObjectID object_id, int error) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaReleaseReply(fbb, to_flat(fbb, object_id), + auto message = CreatePlasmaReleaseReply(fbb, to_flatbuf(fbb, object_id), (PlasmaError) error); fbb.Finish(message); return write_message(sock, MessageType_PlasmaReleaseReply, fbb.GetSize(), @@ -195,7 +159,7 @@ int plasma_send_ReleaseReply(int sock, void plasma_read_ReleaseReply(uint8_t *data, ObjectID *object_id, int *error) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *error = message->error(); } @@ -205,7 +169,7 @@ int plasma_send_DeleteRequest(int sock, protocol_builder *B, ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaDeleteRequest(fbb, to_flat(fbb, object_id)); + auto message = CreatePlasmaDeleteRequest(fbb, to_flatbuf(fbb, object_id)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaDeleteRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -214,7 +178,7 @@ int plasma_send_DeleteRequest(int sock, void plasma_read_DeleteRequest(uint8_t *data, ObjectID *object_id) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); } int plasma_send_DeleteReply(int sock, @@ -222,7 +186,7 @@ int plasma_send_DeleteReply(int sock, ObjectID object_id, int error) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaDeleteReply(fbb, to_flat(fbb, object_id), + auto message = CreatePlasmaDeleteReply(fbb, to_flatbuf(fbb, object_id), (PlasmaError) error); fbb.Finish(message); return write_message(sock, MessageType_PlasmaDeleteReply, fbb.GetSize(), @@ -232,7 +196,7 @@ int plasma_send_DeleteReply(int sock, void plasma_read_DeleteReply(uint8_t *data, ObjectID *object_id, int *error) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *error = message->error(); } @@ -244,7 +208,7 @@ int plasma_send_StatusRequest(int sock, int64_t num_objects) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto message = - CreatePlasmaStatusRequest(fbb, to_flat(fbb, object_ids, num_objects)); + CreatePlasmaStatusRequest(fbb, to_flatbuf(fbb, object_ids, num_objects)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaStatusRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -262,7 +226,7 @@ void plasma_read_StatusRequest(uint8_t *data, DCHECK(data); auto message = flatbuffers::GetRoot(data); for (int64_t i = 0; i < num_objects; ++i) { - object_ids[i] = from_flat(message->object_ids()->Get(i)); + object_ids[i] = from_flatbuf(message->object_ids()->Get(i)); } } @@ -273,7 +237,7 @@ int plasma_send_StatusReply(int sock, int64_t num_objects) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto message = - CreatePlasmaStatusReply(fbb, to_flat(fbb, object_ids, num_objects), + CreatePlasmaStatusReply(fbb, to_flatbuf(fbb, object_ids, num_objects), fbb.CreateVector(object_status, num_objects)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaStatusReply, fbb.GetSize(), @@ -293,7 +257,7 @@ void plasma_read_StatusReply(uint8_t *data, DCHECK(data); auto message = flatbuffers::GetRoot(data); for (int64_t i = 0; i < num_objects; ++i) { - object_ids[i] = from_flat(message->object_ids()->Get(i)); + object_ids[i] = from_flatbuf(message->object_ids()->Get(i)); } for (int64_t i = 0; i < num_objects; ++i) { object_status[i] = message->status()->data()[i]; @@ -306,7 +270,7 @@ int plasma_send_ContainsRequest(int sock, protocol_builder *B, ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaContainsRequest(fbb, to_flat(fbb, object_id)); + auto message = CreatePlasmaContainsRequest(fbb, to_flatbuf(fbb, object_id)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaContainsRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -315,7 +279,7 @@ int plasma_send_ContainsRequest(int sock, void plasma_read_ContainsRequest(uint8_t *data, ObjectID *object_id) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); } int plasma_send_ContainsReply(int sock, @@ -324,7 +288,7 @@ int plasma_send_ContainsReply(int sock, int has_object) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto message = - CreatePlasmaContainsReply(fbb, to_flat(fbb, object_id), has_object); + CreatePlasmaContainsReply(fbb, to_flatbuf(fbb, object_id), has_object); fbb.Finish(message); return write_message(sock, MessageType_PlasmaContainsReply, fbb.GetSize(), fbb.GetBufferPointer()); @@ -335,7 +299,7 @@ void plasma_read_ContainsReply(uint8_t *data, int *has_object) { CHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *has_object = message->has_object(); } @@ -406,7 +370,7 @@ int plasma_send_GetRequest(int sock, int64_t timeout_ms) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto message = CreatePlasmaGetRequest( - fbb, to_flat(fbb, object_ids, num_objects), timeout_ms); + fbb, to_flatbuf(fbb, object_ids, num_objects), timeout_ms); fbb.Finish(message); return write_message(sock, MessageType_PlasmaGetRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -425,7 +389,7 @@ void plasma_read_GetRequest(uint8_t *data, DCHECK(data); auto message = flatbuffers::GetRoot(data); for (int64_t i = 0; i < num_objects; ++i) { - object_ids[i] = from_flat(message->object_ids()->Get(i)); + object_ids[i] = from_flatbuf(message->object_ids()->Get(i)); } *timeout_ms = message->timeout_ms(); } @@ -445,7 +409,7 @@ int plasma_send_GetReply(int sock, object->data_size, object->metadata_offset, object->metadata_size)); } auto message = CreatePlasmaGetReply( - fbb, to_flat(fbb, object_ids, num_objects), + fbb, to_flatbuf(fbb, object_ids, num_objects), fbb.CreateVectorOfStructs(objects.data(), num_objects)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaGetReply, fbb.GetSize(), @@ -459,7 +423,7 @@ void plasma_read_GetReply(uint8_t *data, CHECK(data); auto message = flatbuffers::GetRoot(data); for (int64_t i = 0; i < num_objects; ++i) { - object_ids[i] = from_flat(message->object_ids()->Get(i)); + object_ids[i] = from_flatbuf(message->object_ids()->Get(i)); } for (int64_t i = 0; i < num_objects; ++i) { const PlasmaObjectSpec *object = message->plasma_objects()->Get(i); @@ -480,7 +444,7 @@ int plasma_send_FetchRequest(int sock, int64_t num_objects) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto message = - CreatePlasmaFetchRequest(fbb, to_flat(fbb, object_ids, num_objects)); + CreatePlasmaFetchRequest(fbb, to_flatbuf(fbb, object_ids, num_objects)); fbb.Finish(message); return write_message(sock, MessageType_PlasmaFetchRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -498,7 +462,7 @@ void plasma_read_FetchRequest(uint8_t *data, CHECK(data); auto message = flatbuffers::GetRoot(data); for (int64_t i = 0; i < num_objects; ++i) { - object_ids[i] = from_flat(message->object_ids()->Get(i)); + object_ids[i] = from_flatbuf(message->object_ids()->Get(i)); } } @@ -514,9 +478,9 @@ int plasma_send_WaitRequest(int sock, std::vector> object_request_specs; for (int i = 0; i < num_requests; i++) { - object_request_specs.push_back( - CreateObjectRequestSpec(fbb, to_flat(fbb, object_requests[i].object_id), - object_requests[i].type)); + object_request_specs.push_back(CreateObjectRequestSpec( + fbb, to_flatbuf(fbb, object_requests[i].object_id), + object_requests[i].type)); } auto message = @@ -546,7 +510,7 @@ void plasma_read_WaitRequest(uint8_t *data, CHECK(num_object_ids == message->object_requests()->size()); for (int i = 0; i < num_object_ids; i++) { object_requests[i].object_id = - from_flat(message->object_requests()->Get(i)->object_id()); + from_flatbuf(message->object_requests()->Get(i)->object_id()); object_requests[i].type = message->object_requests()->Get(i)->type(); } } @@ -560,7 +524,7 @@ int plasma_send_WaitReply(int sock, std::vector> object_replies; for (int i = 0; i < num_ready_objects; i++) { object_replies.push_back( - CreateObjectReply(fbb, to_flat(fbb, object_requests[i].object_id), + CreateObjectReply(fbb, to_flatbuf(fbb, object_requests[i].object_id), object_requests[i].status)); } @@ -581,7 +545,7 @@ void plasma_read_WaitReply(uint8_t *data, *num_ready_objects = message->num_ready_objects(); for (int i = 0; i < *num_ready_objects; i++) { object_requests[i].object_id = - from_flat(message->object_requests()->Get(i)->object_id()); + from_flatbuf(message->object_requests()->Get(i)->object_id()); object_requests[i].status = message->object_requests()->Get(i)->status(); } } @@ -606,7 +570,7 @@ int plasma_send_DataRequest(int sock, flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); auto addr = fbb.CreateString((char *) address, strlen(address)); auto message = - CreatePlasmaDataRequest(fbb, to_flat(fbb, object_id), addr, port); + CreatePlasmaDataRequest(fbb, to_flatbuf(fbb, object_id), addr, port); fbb.Finish(message); return write_message(sock, MessageType_PlasmaDataRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -619,7 +583,7 @@ void plasma_read_DataRequest(uint8_t *data, DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(message->object_id()->size() == sizeof(object_id->id)); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *address = strdup(message->address()->c_str()); *port = message->port(); } @@ -630,7 +594,7 @@ int plasma_send_DataReply(int sock, int64_t object_size, int64_t metadata_size) { flatbuffers::FlatBufferBuilder fbb(FLATBUFFER_BUILDER_DEFAULT_SIZE); - auto message = CreatePlasmaDataReply(fbb, to_flat(fbb, object_id), + auto message = CreatePlasmaDataReply(fbb, to_flatbuf(fbb, object_id), object_size, metadata_size); fbb.Finish(message); return write_message(sock, MessageType_PlasmaDataReply, fbb.GetSize(), @@ -643,7 +607,7 @@ void plasma_read_DataReply(uint8_t *data, int64_t *metadata_size) { DCHECK(data); auto message = flatbuffers::GetRoot(data); - *object_id = from_flat(message->object_id()); + *object_id = from_flatbuf(message->object_id()); *object_size = (int64_t) message->object_size(); *metadata_size = (int64_t) message->metadata_size(); }