mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
Change type naming convention. (#315)
* Rename object_id -> ObjectID. * Rename ray_logger -> RayLogger. * rename task_id -> TaskID, actor_id -> ActorID, function_id -> FunctionID * Rename plasma_store_info -> PlasmaStoreInfo. * Rename plasma_store_state -> PlasmaStoreState. * Rename plasma_object -> PlasmaObject. * Rename object_request -> ObjectRequests. * Rename eviction_state -> EvictionState. * Bug fix. * rename db_handle -> DBHandle * Rename local_scheduler_state -> LocalSchedulerState. * rename db_client_id -> DBClientID * rename task -> Task * make redis.c C++ compatible * Rename scheduling_algorithm_state -> SchedulingAlgorithmState. * Rename plasma_connection -> PlasmaConnection. * Rename client_connection -> ClientConnection. * Fixes from rebase. * Rename local_scheduler_client -> LocalSchedulerClient. * Rename object_buffer -> ObjectBuffer. * Rename client -> Client. * Rename notification_queue -> NotificationQueue. * Rename object_get_requests -> ObjectGetRequests. * Rename get_request -> GetRequest. * Rename object_info -> ObjectInfo. * Rename scheduler_object_info -> SchedulerObjectInfo. * Rename local_scheduler -> LocalScheduler and some fixes. * Rename local_scheduler_info -> LocalSchedulerInfo. * Rename global_scheduler_state -> GlobalSchedulerState. * Rename global_scheduler_policy_state -> GlobalSchedulerPolicyState. * Rename object_size_entry -> ObjectSizeEntry. * Rename aux_address_entry -> AuxAddressEntry. * Rename various ID helper methods. * Rename Task helper methods. * Rename db_client_cache_entry -> DBClientCacheEntry. * Rename local_actor_info -> LocalActorInfo. * Rename actor_info -> ActorInfo. * Rename retry_info -> RetryInfo. * Rename actor_notification_table_subscribe_data -> ActorNotificationTableSubscribeData. * Rename local_scheduler_table_send_info_data -> LocalSchedulerTableSendInfoData. * Rename table_callback_data -> TableCallbackData. * Rename object_info_subscribe_data -> ObjectInfoSubscribeData. * Rename local_scheduler_table_subscribe_data -> LocalSchedulerTableSubscribeData. * Rename more redis call data structures. * Rename photon_conn PhotonConnection. * Rename photon_mock -> PhotonMock. * Fix formatting errors.
This commit is contained in:
parent
be1618f041
commit
a30eed452e
64 changed files with 2020 additions and 2115 deletions
|
@ -9,39 +9,39 @@
|
||||||
#include "io.h"
|
#include "io.h"
|
||||||
|
|
||||||
/* This is used to define the array of object IDs. */
|
/* This is used to define the array of object IDs. */
|
||||||
const UT_icd object_id_icd = {sizeof(object_id), NULL, NULL, NULL};
|
const UT_icd object_id_icd = {sizeof(ObjectID), NULL, NULL, NULL};
|
||||||
|
|
||||||
const unique_id NIL_ID = {{255, 255, 255, 255, 255, 255, 255, 255, 255, 255,
|
const UniqueID NIL_ID = {{255, 255, 255, 255, 255, 255, 255, 255, 255, 255,
|
||||||
255, 255, 255, 255, 255, 255, 255, 255, 255, 255}};
|
255, 255, 255, 255, 255, 255, 255, 255, 255, 255}};
|
||||||
|
|
||||||
const unsigned char NIL_DIGEST[DIGEST_SIZE] = {0};
|
const unsigned char NIL_DIGEST[DIGEST_SIZE] = {0};
|
||||||
|
|
||||||
unique_id globally_unique_id(void) {
|
UniqueID globally_unique_id(void) {
|
||||||
/* Use /dev/urandom for "real" randomness. */
|
/* Use /dev/urandom for "real" randomness. */
|
||||||
int fd;
|
int fd;
|
||||||
int const flags = 0 /* for Windows compatibility */;
|
int const flags = 0 /* for Windows compatibility */;
|
||||||
if ((fd = open("/dev/urandom", O_RDONLY, flags)) == -1) {
|
if ((fd = open("/dev/urandom", O_RDONLY, flags)) == -1) {
|
||||||
LOG_ERROR("Could not generate random number");
|
LOG_ERROR("Could not generate random number");
|
||||||
}
|
}
|
||||||
unique_id result;
|
UniqueID result;
|
||||||
CHECK(read_bytes(fd, &result.id[0], UNIQUE_ID_SIZE) >= 0);
|
CHECK(read_bytes(fd, &result.id[0], UNIQUE_ID_SIZE) >= 0);
|
||||||
close(fd);
|
close(fd);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool object_ids_equal(object_id first_id, object_id second_id) {
|
bool ObjectID_equal(ObjectID first_id, ObjectID second_id) {
|
||||||
return UNIQUE_ID_EQ(first_id, second_id);
|
return UNIQUE_ID_EQ(first_id, second_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool object_id_is_nil(object_id id) {
|
bool ObjectID_is_nil(ObjectID id) {
|
||||||
return object_ids_equal(id, NIL_OBJECT_ID);
|
return ObjectID_equal(id, NIL_OBJECT_ID);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool db_client_ids_equal(db_client_id first_id, db_client_id second_id) {
|
bool DBClientID_equal(DBClientID first_id, DBClientID second_id) {
|
||||||
return UNIQUE_ID_EQ(first_id, second_id);
|
return UNIQUE_ID_EQ(first_id, second_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
char *object_id_to_string(object_id obj_id, char *id_string, int id_length) {
|
char *ObjectID_to_string(ObjectID obj_id, char *id_string, int id_length) {
|
||||||
CHECK(id_length >= ID_STRING_SIZE);
|
CHECK(id_length >= ID_STRING_SIZE);
|
||||||
static const char hex[] = "0123456789abcdef";
|
static const char hex[] = "0123456789abcdef";
|
||||||
char *buf = id_string;
|
char *buf = id_string;
|
||||||
|
|
|
@ -121,18 +121,18 @@
|
||||||
|
|
||||||
#define IS_NIL_ID(id) UNIQUE_ID_EQ(id, NIL_ID)
|
#define IS_NIL_ID(id) UNIQUE_ID_EQ(id, NIL_ID)
|
||||||
|
|
||||||
typedef struct { unsigned char id[UNIQUE_ID_SIZE]; } unique_id;
|
typedef struct { unsigned char id[UNIQUE_ID_SIZE]; } UniqueID;
|
||||||
|
|
||||||
extern const UT_icd object_id_icd;
|
extern const UT_icd object_id_icd;
|
||||||
|
|
||||||
extern const unique_id NIL_ID;
|
extern const UniqueID NIL_ID;
|
||||||
|
|
||||||
/* Generate a globally unique ID. */
|
/* Generate a globally unique ID. */
|
||||||
unique_id globally_unique_id(void);
|
UniqueID globally_unique_id(void);
|
||||||
|
|
||||||
#define NIL_OBJECT_ID NIL_ID
|
#define NIL_OBJECT_ID NIL_ID
|
||||||
|
|
||||||
typedef unique_id object_id;
|
typedef UniqueID ObjectID;
|
||||||
|
|
||||||
#define ID_STRING_SIZE (2 * UNIQUE_ID_SIZE + 1)
|
#define ID_STRING_SIZE (2 * UNIQUE_ID_SIZE + 1)
|
||||||
|
|
||||||
|
@ -147,7 +147,7 @@ typedef unique_id object_id;
|
||||||
* string.
|
* string.
|
||||||
* @param id_length The length of the id_string buffer.
|
* @param id_length The length of the id_string buffer.
|
||||||
*/
|
*/
|
||||||
char *object_id_to_string(object_id obj_id, char *id_string, int id_length);
|
char *ObjectID_to_string(ObjectID obj_id, char *id_string, int id_length);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare two object IDs.
|
* Compare two object IDs.
|
||||||
|
@ -156,7 +156,7 @@ char *object_id_to_string(object_id obj_id, char *id_string, int id_length);
|
||||||
* @param second_id The first object ID to compare.
|
* @param second_id The first object ID to compare.
|
||||||
* @return True if the object IDs are the same and false otherwise.
|
* @return True if the object IDs are the same and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool object_ids_equal(object_id first_id, object_id second_id);
|
bool ObjectID_equal(ObjectID first_id, ObjectID second_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare a object ID to the nil ID.
|
* Compare a object ID to the nil ID.
|
||||||
|
@ -164,9 +164,9 @@ bool object_ids_equal(object_id first_id, object_id second_id);
|
||||||
* @param id The object ID to compare to nil.
|
* @param id The object ID to compare to nil.
|
||||||
* @return True if the object ID is equal to nil.
|
* @return True if the object ID is equal to nil.
|
||||||
*/
|
*/
|
||||||
bool object_id_is_nil(object_id id);
|
bool ObjectID_is_nil(ObjectID id);
|
||||||
|
|
||||||
typedef unique_id db_client_id;
|
typedef UniqueID DBClientID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare two db client IDs.
|
* Compare two db client IDs.
|
||||||
|
@ -175,7 +175,7 @@ typedef unique_id db_client_id;
|
||||||
* @param second_id The first db client ID to compare.
|
* @param second_id The first db client ID to compare.
|
||||||
* @return True if the db client IDs are the same and false otherwise.
|
* @return True if the db client IDs are the same and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool db_client_ids_equal(db_client_id first_id, db_client_id second_id);
|
bool DBClientID_equal(DBClientID first_id, DBClientID second_id);
|
||||||
|
|
||||||
#define MAX(x, y) ((x) >= (y) ? (x) : (y))
|
#define MAX(x, y) ((x) >= (y) ? (x) : (y))
|
||||||
#define MIN(x, y) ((x) <= (y) ? (x) : (y))
|
#define MIN(x, y) ((x) <= (y) ? (x) : (y))
|
||||||
|
|
|
@ -37,7 +37,7 @@ int bind_inet_sock(const int port, bool shall_listen) {
|
||||||
close(socket_fd);
|
close(socket_fd);
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
int *const pon = (char const *) &on;
|
int *const pon = (int *const) & on;
|
||||||
if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, pon, sizeof(on)) < 0) {
|
if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, pon, sizeof(on)) < 0) {
|
||||||
LOG_ERROR("setsockopt failed for port %d", port);
|
LOG_ERROR("setsockopt failed for port %d", port);
|
||||||
close(socket_fd);
|
close(socket_fd);
|
||||||
|
@ -302,7 +302,7 @@ void read_message(int fd, int64_t *type, int64_t *length, uint8_t **bytes) {
|
||||||
if (closed) {
|
if (closed) {
|
||||||
goto disconnected;
|
goto disconnected;
|
||||||
}
|
}
|
||||||
*bytes = malloc(*length * sizeof(uint8_t));
|
*bytes = (uint8_t *) malloc(*length * sizeof(uint8_t));
|
||||||
closed = read_bytes(fd, *bytes, *length);
|
closed = read_bytes(fd, *bytes, *length);
|
||||||
if (closed) {
|
if (closed) {
|
||||||
free(*bytes);
|
free(*bytes);
|
||||||
|
|
|
@ -35,7 +35,7 @@ void init_pickle_module(void) {
|
||||||
|
|
||||||
/* Define the PyObjectID class. */
|
/* Define the PyObjectID class. */
|
||||||
|
|
||||||
int PyStringToUniqueID(PyObject *object, object_id *object_id) {
|
int PyStringToUniqueID(PyObject *object, ObjectID *object_id) {
|
||||||
if (PyBytes_Check(object)) {
|
if (PyBytes_Check(object)) {
|
||||||
memcpy(&object_id->id[0], PyBytes_AsString(object), UNIQUE_ID_SIZE);
|
memcpy(&object_id->id[0], PyBytes_AsString(object), UNIQUE_ID_SIZE);
|
||||||
return 1;
|
return 1;
|
||||||
|
@ -45,7 +45,7 @@ int PyStringToUniqueID(PyObject *object, object_id *object_id) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
int PyObjectToUniqueID(PyObject *object, object_id *objectid) {
|
int PyObjectToUniqueID(PyObject *object, ObjectID *objectid) {
|
||||||
if (PyObject_IsInstance(object, (PyObject *) &PyObjectIDType)) {
|
if (PyObject_IsInstance(object, (PyObject *) &PyObjectIDType)) {
|
||||||
*objectid = ((PyObjectID *) object)->object_id;
|
*objectid = ((PyObjectID *) object)->object_id;
|
||||||
return 1;
|
return 1;
|
||||||
|
@ -61,7 +61,7 @@ static int PyObjectID_init(PyObjectID *self, PyObject *args, PyObject *kwds) {
|
||||||
if (!PyArg_ParseTuple(args, "s#", &data, &size)) {
|
if (!PyArg_ParseTuple(args, "s#", &data, &size)) {
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
if (size != sizeof(object_id)) {
|
if (size != sizeof(ObjectID)) {
|
||||||
PyErr_SetString(CommonError,
|
PyErr_SetString(CommonError,
|
||||||
"ObjectID: object id string needs to have length 20");
|
"ObjectID: object id string needs to have length 20");
|
||||||
return -1;
|
return -1;
|
||||||
|
@ -71,7 +71,7 @@ static int PyObjectID_init(PyObjectID *self, PyObject *args, PyObject *kwds) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Create a PyObjectID from C. */
|
/* Create a PyObjectID from C. */
|
||||||
PyObject *PyObjectID_make(object_id object_id) {
|
PyObject *PyObjectID_make(ObjectID object_id) {
|
||||||
PyObjectID *result = PyObject_New(PyObjectID, &PyObjectIDType);
|
PyObjectID *result = PyObject_New(PyObjectID, &PyObjectIDType);
|
||||||
result = (PyObjectID *) PyObject_Init((PyObject *) result, &PyObjectIDType);
|
result = (PyObjectID *) PyObject_Init((PyObject *) result, &PyObjectIDType);
|
||||||
result->object_id = object_id;
|
result->object_id = object_id;
|
||||||
|
@ -136,7 +136,7 @@ static PyObject *PyObjectID_id(PyObject *self) {
|
||||||
static PyObject *PyObjectID_hex(PyObject *self) {
|
static PyObject *PyObjectID_hex(PyObject *self) {
|
||||||
PyObjectID *s = (PyObjectID *) self;
|
PyObjectID *s = (PyObjectID *) self;
|
||||||
char hex_id[ID_STRING_SIZE];
|
char hex_id[ID_STRING_SIZE];
|
||||||
object_id_to_string(s->object_id, hex_id, ID_STRING_SIZE);
|
ObjectID_to_string(s->object_id, hex_id, ID_STRING_SIZE);
|
||||||
PyObject *result = PyUnicode_FromString(hex_id);
|
PyObject *result = PyUnicode_FromString(hex_id);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -157,14 +157,12 @@ static PyObject *PyObjectID_richcompare(PyObjectID *self,
|
||||||
result = Py_NotImplemented;
|
result = Py_NotImplemented;
|
||||||
break;
|
break;
|
||||||
case Py_EQ:
|
case Py_EQ:
|
||||||
result = object_ids_equal(self->object_id, other_id->object_id)
|
result = ObjectID_equal(self->object_id, other_id->object_id) ? Py_True
|
||||||
? Py_True
|
: Py_False;
|
||||||
: Py_False;
|
|
||||||
break;
|
break;
|
||||||
case Py_NE:
|
case Py_NE:
|
||||||
result = !object_ids_equal(self->object_id, other_id->object_id)
|
result = !ObjectID_equal(self->object_id, other_id->object_id) ? Py_True
|
||||||
? Py_True
|
: Py_False;
|
||||||
: Py_False;
|
|
||||||
break;
|
break;
|
||||||
case Py_GT:
|
case Py_GT:
|
||||||
result = Py_NotImplemented;
|
result = Py_NotImplemented;
|
||||||
|
@ -190,7 +188,7 @@ static long PyObjectID_hash(PyObjectID *self) {
|
||||||
|
|
||||||
static PyObject *PyObjectID_repr(PyObjectID *self) {
|
static PyObject *PyObjectID_repr(PyObjectID *self) {
|
||||||
char hex_id[ID_STRING_SIZE];
|
char hex_id[ID_STRING_SIZE];
|
||||||
object_id_to_string(self->object_id, hex_id, ID_STRING_SIZE);
|
ObjectID_to_string(self->object_id, hex_id, ID_STRING_SIZE);
|
||||||
UT_string *repr;
|
UT_string *repr;
|
||||||
utstring_new(repr);
|
utstring_new(repr);
|
||||||
utstring_printf(repr, "ObjectID(%s)", hex_id);
|
utstring_printf(repr, "ObjectID(%s)", hex_id);
|
||||||
|
@ -264,13 +262,13 @@ PyTypeObject PyObjectIDType = {
|
||||||
|
|
||||||
static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) {
|
static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) {
|
||||||
/* ID of the driver that this task originates from. */
|
/* ID of the driver that this task originates from. */
|
||||||
unique_id driver_id;
|
UniqueID driver_id;
|
||||||
/* ID of the actor this task should run on. */
|
/* ID of the actor this task should run on. */
|
||||||
unique_id actor_id = NIL_ACTOR_ID;
|
UniqueID actor_id = NIL_ACTOR_ID;
|
||||||
/* How many tasks have been launched on the actor so far? */
|
/* How many tasks have been launched on the actor so far? */
|
||||||
int actor_counter = 0;
|
int actor_counter = 0;
|
||||||
/* ID of the function this task executes. */
|
/* ID of the function this task executes. */
|
||||||
function_id function_id;
|
FunctionID function_id;
|
||||||
/* Arguments of the task (can be PyObjectIDs or Python values). */
|
/* Arguments of the task (can be PyObjectIDs or Python values). */
|
||||||
PyObject *arguments;
|
PyObject *arguments;
|
||||||
/* Array of pointers to string representations of pass-by-value args. */
|
/* Array of pointers to string representations of pass-by-value args. */
|
||||||
|
@ -278,7 +276,7 @@ static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) {
|
||||||
utarray_new(val_repr_ptrs, &ut_ptr_icd);
|
utarray_new(val_repr_ptrs, &ut_ptr_icd);
|
||||||
int num_returns;
|
int num_returns;
|
||||||
/* The ID of the task that called this task. */
|
/* The ID of the task that called this task. */
|
||||||
task_id parent_task_id;
|
TaskID parent_task_id;
|
||||||
/* The number of tasks that the parent task has called prior to this one. */
|
/* The number of tasks that the parent task has called prior to this one. */
|
||||||
int parent_counter;
|
int parent_counter;
|
||||||
/* Resource vector of the required resources to execute this task. */
|
/* Resource vector of the required resources to execute this task. */
|
||||||
|
@ -353,22 +351,22 @@ static void PyTask_dealloc(PyTask *self) {
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *PyTask_function_id(PyObject *self) {
|
static PyObject *PyTask_function_id(PyObject *self) {
|
||||||
function_id function_id = task_function(((PyTask *) self)->spec);
|
FunctionID function_id = task_function(((PyTask *) self)->spec);
|
||||||
return PyObjectID_make(function_id);
|
return PyObjectID_make(function_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *PyTask_actor_id(PyObject *self) {
|
static PyObject *PyTask_actor_id(PyObject *self) {
|
||||||
actor_id actor_id = task_spec_actor_id(((PyTask *) self)->spec);
|
ActorID actor_id = task_spec_actor_id(((PyTask *) self)->spec);
|
||||||
return PyObjectID_make(actor_id);
|
return PyObjectID_make(actor_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *PyTask_driver_id(PyObject *self) {
|
static PyObject *PyTask_driver_id(PyObject *self) {
|
||||||
unique_id driver_id = task_spec_driver_id(((PyTask *) self)->spec);
|
UniqueID driver_id = task_spec_driver_id(((PyTask *) self)->spec);
|
||||||
return PyObjectID_make(driver_id);
|
return PyObjectID_make(driver_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *PyTask_task_id(PyObject *self) {
|
static PyObject *PyTask_task_id(PyObject *self) {
|
||||||
task_id task_id = task_spec_id(((PyTask *) self)->spec);
|
TaskID task_id = task_spec_id(((PyTask *) self)->spec);
|
||||||
return PyObjectID_make(task_id);
|
return PyObjectID_make(task_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -378,7 +376,7 @@ static PyObject *PyTask_arguments(PyObject *self) {
|
||||||
PyObject *arg_list = PyList_New((Py_ssize_t) num_args);
|
PyObject *arg_list = PyList_New((Py_ssize_t) num_args);
|
||||||
for (int i = 0; i < num_args; ++i) {
|
for (int i = 0; i < num_args; ++i) {
|
||||||
if (task_arg_type(task, i) == ARG_BY_REF) {
|
if (task_arg_type(task, i) == ARG_BY_REF) {
|
||||||
object_id object_id = task_arg_id(task, i);
|
ObjectID object_id = task_arg_id(task, i);
|
||||||
PyList_SetItem(arg_list, i, PyObjectID_make(object_id));
|
PyList_SetItem(arg_list, i, PyObjectID_make(object_id));
|
||||||
} else {
|
} else {
|
||||||
CHECK(pickle_module != NULL);
|
CHECK(pickle_module != NULL);
|
||||||
|
@ -410,7 +408,7 @@ static PyObject *PyTask_returns(PyObject *self) {
|
||||||
int64_t num_returns = task_num_returns(task);
|
int64_t num_returns = task_num_returns(task);
|
||||||
PyObject *return_id_list = PyList_New((Py_ssize_t) num_returns);
|
PyObject *return_id_list = PyList_New((Py_ssize_t) num_returns);
|
||||||
for (int i = 0; i < num_returns; ++i) {
|
for (int i = 0; i < num_returns; ++i) {
|
||||||
object_id object_id = task_return(task, i);
|
ObjectID object_id = task_return(task, i);
|
||||||
PyList_SetItem(return_id_list, i, PyObjectID_make(object_id));
|
PyList_SetItem(return_id_list, i, PyObjectID_make(object_id));
|
||||||
}
|
}
|
||||||
return return_id_list;
|
return return_id_list;
|
||||||
|
@ -569,11 +567,11 @@ PyObject *check_simple_value(PyObject *self, PyObject *args) {
|
||||||
|
|
||||||
PyObject *compute_put_id(PyObject *self, PyObject *args) {
|
PyObject *compute_put_id(PyObject *self, PyObject *args) {
|
||||||
int put_index;
|
int put_index;
|
||||||
task_id task_id;
|
TaskID task_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&i", &PyObjectToUniqueID, &task_id,
|
if (!PyArg_ParseTuple(args, "O&i", &PyObjectToUniqueID, &task_id,
|
||||||
&put_index)) {
|
&put_index)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
object_id put_id = task_compute_put_id(task_id, put_index);
|
ObjectID put_id = task_compute_put_id(task_id, put_index);
|
||||||
return PyObjectID_make(put_id);
|
return PyObjectID_make(put_id);
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,7 +13,7 @@ extern PyObject *CommonError;
|
||||||
// clang-format off
|
// clang-format off
|
||||||
typedef struct {
|
typedef struct {
|
||||||
PyObject_HEAD
|
PyObject_HEAD
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
} PyObjectID;
|
} PyObjectID;
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
|
@ -33,11 +33,11 @@ extern PyObject *pickle_loads;
|
||||||
|
|
||||||
void init_pickle_module(void);
|
void init_pickle_module(void);
|
||||||
|
|
||||||
int PyStringToUniqueID(PyObject *object, object_id *object_id);
|
int PyStringToUniqueID(PyObject *object, ObjectID *object_id);
|
||||||
|
|
||||||
int PyObjectToUniqueID(PyObject *object, object_id *objectid);
|
int PyObjectToUniqueID(PyObject *object, ObjectID *objectid);
|
||||||
|
|
||||||
PyObject *PyObjectID_make(object_id object_id);
|
PyObject *PyObjectID_make(ObjectID object_id);
|
||||||
|
|
||||||
PyObject *check_simple_value(PyObject *self, PyObject *args);
|
PyObject *check_simple_value(PyObject *self, PyObject *args);
|
||||||
|
|
||||||
|
|
|
@ -14,7 +14,7 @@ static const char *log_levels[5] = {"DEBUG", "INFO", "WARN", "ERROR", "FATAL"};
|
||||||
static const char *log_fmt =
|
static const char *log_fmt =
|
||||||
"HMSET log:%s:%s log_level %s event_type %s message %s timestamp %s";
|
"HMSET log:%s:%s log_level %s event_type %s message %s timestamp %s";
|
||||||
|
|
||||||
struct ray_logger_impl {
|
struct RayLoggerImpl {
|
||||||
/* String that identifies this client type. */
|
/* String that identifies this client type. */
|
||||||
const char *client_type;
|
const char *client_type;
|
||||||
/* Suppress all log messages below this level. */
|
/* Suppress all log messages below this level. */
|
||||||
|
@ -26,11 +26,11 @@ struct ray_logger_impl {
|
||||||
void *conn;
|
void *conn;
|
||||||
};
|
};
|
||||||
|
|
||||||
ray_logger *init_ray_logger(const char *client_type,
|
RayLogger *RayLogger_init(const char *client_type,
|
||||||
int log_level,
|
int log_level,
|
||||||
int is_direct,
|
int is_direct,
|
||||||
void *conn) {
|
void *conn) {
|
||||||
ray_logger *logger = malloc(sizeof(ray_logger));
|
RayLogger *logger = (RayLogger *) malloc(sizeof(RayLogger));
|
||||||
logger->client_type = client_type;
|
logger->client_type = client_type;
|
||||||
logger->log_level = log_level;
|
logger->log_level = log_level;
|
||||||
logger->is_direct = is_direct;
|
logger->is_direct = is_direct;
|
||||||
|
@ -38,14 +38,14 @@ ray_logger *init_ray_logger(const char *client_type,
|
||||||
return logger;
|
return logger;
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_ray_logger(ray_logger *logger) {
|
void RayLogger_free(RayLogger *logger) {
|
||||||
free(logger);
|
free(logger);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ray_log(ray_logger *logger,
|
void RayLogger_log(RayLogger *logger,
|
||||||
int log_level,
|
int log_level,
|
||||||
const char *event_type,
|
const char *event_type,
|
||||||
const char *message) {
|
const char *message) {
|
||||||
if (log_level < logger->log_level) {
|
if (log_level < logger->log_level) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -65,7 +65,7 @@ void ray_log(ray_logger *logger,
|
||||||
log_levels[log_level], event_type, message,
|
log_levels[log_level], event_type, message,
|
||||||
utstring_body(timestamp));
|
utstring_body(timestamp));
|
||||||
if (logger->is_direct) {
|
if (logger->is_direct) {
|
||||||
db_handle *db = (db_handle *) logger->conn;
|
DBHandle *db = (DBHandle *) logger->conn;
|
||||||
/* Fill in the client ID and send the message to Redis. */
|
/* Fill in the client ID and send the message to Redis. */
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, NULL, NULL, utstring_body(formatted_message),
|
db->context, NULL, NULL, utstring_body(formatted_message),
|
||||||
|
@ -83,11 +83,11 @@ void ray_log(ray_logger *logger,
|
||||||
utstring_free(timestamp);
|
utstring_free(timestamp);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ray_log_event(db_handle *db,
|
void RayLogger_log_event(DBHandle *db,
|
||||||
uint8_t *key,
|
uint8_t *key,
|
||||||
int64_t key_length,
|
int64_t key_length,
|
||||||
uint8_t *value,
|
uint8_t *value,
|
||||||
int64_t value_length) {
|
int64_t value_length) {
|
||||||
int status = redisAsyncCommand(db->context, NULL, NULL, "RPUSH %b %b", key,
|
int status = redisAsyncCommand(db->context, NULL, NULL, "RPUSH %b %b", key,
|
||||||
key_length, value, value_length);
|
key_length, value, value_length);
|
||||||
if ((status == REDIS_ERR) || db->context->err) {
|
if ((status == REDIS_ERR) || db->context->err) {
|
||||||
|
|
|
@ -15,28 +15,28 @@
|
||||||
|
|
||||||
#include "state/db.h"
|
#include "state/db.h"
|
||||||
|
|
||||||
typedef struct ray_logger_impl ray_logger;
|
typedef struct RayLoggerImpl RayLogger;
|
||||||
|
|
||||||
/* Initialize a Ray logger for the given client type and logging level. If the
|
/* Initialize a Ray logger for the given client type and logging level. If the
|
||||||
* is_direct flag is set, the logger will treat the given connection as a
|
* is_direct flag is set, the logger will treat the given connection as a
|
||||||
* direct connection to the log. Otherwise, it will treat it as a socket to
|
* direct connection to the log. Otherwise, it will treat it as a socket to
|
||||||
* another process with a connection to the log.
|
* another process with a connection to the log.
|
||||||
* NOTE: User is responsible for freeing the returned logger. */
|
* NOTE: User is responsible for freeing the returned logger. */
|
||||||
ray_logger *init_ray_logger(const char *client_type,
|
RayLogger *RayLogger_init(const char *client_type,
|
||||||
int log_level,
|
int log_level,
|
||||||
int is_direct,
|
int is_direct,
|
||||||
void *conn);
|
void *conn);
|
||||||
|
|
||||||
/* Free the logger. This does not free the connection to the log. */
|
/* Free the logger. This does not free the connection to the log. */
|
||||||
void free_ray_logger(ray_logger *logger);
|
void RayLogger_free(RayLogger *logger);
|
||||||
|
|
||||||
/* Log an event at the given log level with the given event_type.
|
/* Log an event at the given log level with the given event_type.
|
||||||
* NOTE: message cannot contain spaces! JSON format is recommended.
|
* NOTE: message cannot contain spaces! JSON format is recommended.
|
||||||
* TODO: Support spaces in messages. */
|
* TODO: Support spaces in messages. */
|
||||||
void ray_log(ray_logger *logger,
|
void RayLogger_log(RayLogger *logger,
|
||||||
int log_level,
|
int log_level,
|
||||||
const char *event_type,
|
const char *event_type,
|
||||||
const char *message);
|
const char *message);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Log an event to the event log.
|
* Log an event to the event log.
|
||||||
|
@ -48,10 +48,10 @@ void ray_log(ray_logger *logger,
|
||||||
* @param value_length The length of the value.
|
* @param value_length The length of the value.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void ray_log_event(db_handle *db,
|
void RayLogger_log_event(DBHandle *db,
|
||||||
uint8_t *key,
|
uint8_t *key,
|
||||||
int64_t key_length,
|
int64_t key_length,
|
||||||
uint8_t *value,
|
uint8_t *value,
|
||||||
int64_t value_length);
|
int64_t value_length);
|
||||||
|
|
||||||
#endif /* LOGGING_H */
|
#endif /* LOGGING_H */
|
||||||
|
|
|
@ -9,13 +9,13 @@
|
||||||
* Object information data structure.
|
* Object information data structure.
|
||||||
*/
|
*/
|
||||||
typedef struct {
|
typedef struct {
|
||||||
object_id obj_id;
|
ObjectID obj_id;
|
||||||
int64_t data_size;
|
int64_t data_size;
|
||||||
int64_t metadata_size;
|
int64_t metadata_size;
|
||||||
int64_t create_time;
|
int64_t create_time;
|
||||||
int64_t construct_duration;
|
int64_t construct_duration;
|
||||||
unsigned char digest[DIGEST_SIZE];
|
unsigned char digest[DIGEST_SIZE];
|
||||||
bool is_deletion;
|
bool is_deletion;
|
||||||
} object_info;
|
} ObjectInfo;
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
|
@ -2,12 +2,12 @@
|
||||||
#include "redis.h"
|
#include "redis.h"
|
||||||
|
|
||||||
void actor_notification_table_subscribe(
|
void actor_notification_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
actor_notification_table_subscribe_callback subscribe_callback,
|
actor_notification_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry) {
|
RetryInfo *retry) {
|
||||||
actor_notification_table_subscribe_data *sub_data =
|
ActorNotificationTableSubscribeData *sub_data =
|
||||||
malloc(sizeof(actor_notification_table_subscribe_data));
|
malloc(sizeof(ActorNotificationTableSubscribeData));
|
||||||
sub_data->subscribe_callback = subscribe_callback;
|
sub_data->subscribe_callback = subscribe_callback;
|
||||||
sub_data->subscribe_context = subscribe_context;
|
sub_data->subscribe_context = subscribe_context;
|
||||||
|
|
||||||
|
|
|
@ -7,17 +7,17 @@
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the actor. */
|
/** The ID of the actor. */
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** The ID of the local scheduler that is responsible for the actor. */
|
/** The ID of the local scheduler that is responsible for the actor. */
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
} actor_info;
|
} ActorInfo;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Subscribing to the actor notification table ====
|
* ==== Subscribing to the actor notification table ====
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* Callback for subscribing to the local scheduler table. */
|
/* Callback for subscribing to the local scheduler table. */
|
||||||
typedef void (*actor_notification_table_subscribe_callback)(actor_info info,
|
typedef void (*actor_notification_table_subscribe_callback)(ActorInfo info,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,16 +32,16 @@ typedef void (*actor_notification_table_subscribe_callback)(actor_info info,
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void actor_notification_table_subscribe(
|
void actor_notification_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
actor_notification_table_subscribe_callback subscribe_callback,
|
actor_notification_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry);
|
RetryInfo *retry);
|
||||||
|
|
||||||
/* Data that is needed to register local scheduler table subscribe callbacks
|
/* Data that is needed to register local scheduler table subscribe callbacks
|
||||||
* with the state database. */
|
* with the state database. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
actor_notification_table_subscribe_callback subscribe_callback;
|
actor_notification_table_subscribe_callback subscribe_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} actor_notification_table_subscribe_data;
|
} ActorNotificationTableSubscribeData;
|
||||||
|
|
||||||
#endif /* ACTOR_NOTIFICATION_TABLE_H */
|
#endif /* ACTOR_NOTIFICATION_TABLE_H */
|
||||||
|
|
|
@ -4,7 +4,7 @@
|
||||||
#include "common.h"
|
#include "common.h"
|
||||||
#include "event_loop.h"
|
#include "event_loop.h"
|
||||||
|
|
||||||
typedef struct db_handle db_handle;
|
typedef struct DBHandle DBHandle;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Connect to the global system store.
|
* Connect to the global system store.
|
||||||
|
@ -21,12 +21,12 @@ typedef struct db_handle db_handle;
|
||||||
* @return This returns a handle to the database, which must be freed with
|
* @return This returns a handle to the database, which must be freed with
|
||||||
* db_disconnect after use.
|
* db_disconnect after use.
|
||||||
*/
|
*/
|
||||||
db_handle *db_connect(const char *db_address,
|
DBHandle *db_connect(const char *db_address,
|
||||||
int db_port,
|
int db_port,
|
||||||
const char *client_type,
|
const char *client_type,
|
||||||
const char *node_ip_address,
|
const char *node_ip_address,
|
||||||
int num_args,
|
int num_args,
|
||||||
const char **args);
|
const char **args);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Attach global system store connection to an event loop. Callbacks from
|
* Attach global system store connection to an event loop. Callbacks from
|
||||||
|
@ -38,7 +38,7 @@ db_handle *db_connect(const char *db_address,
|
||||||
* reattached to the loop.
|
* reattached to the loop.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void db_attach(db_handle *db, event_loop *loop, bool reattach);
|
void db_attach(DBHandle *db, event_loop *loop, bool reattach);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Disconnect from the global system store.
|
* Disconnect from the global system store.
|
||||||
|
@ -46,7 +46,7 @@ void db_attach(db_handle *db, event_loop *loop, bool reattach);
|
||||||
* @param db The database connection to close and clean up.
|
* @param db The database connection to close and clean up.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void db_disconnect(db_handle *db);
|
void db_disconnect(DBHandle *db);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the db client ID.
|
* Returns the db client ID.
|
||||||
|
@ -54,6 +54,6 @@ void db_disconnect(db_handle *db);
|
||||||
* @param db The handle to the database.
|
* @param db The handle to the database.
|
||||||
* @returns int The db client ID for this connection to the database.
|
* @returns int The db client ID for this connection to the database.
|
||||||
*/
|
*/
|
||||||
db_client_id get_db_client_id(db_handle *db);
|
DBClientID get_db_client_id(DBHandle *db);
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
|
@ -2,14 +2,14 @@
|
||||||
#include "redis.h"
|
#include "redis.h"
|
||||||
|
|
||||||
void db_client_table_subscribe(
|
void db_client_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
db_client_table_subscribe_callback subscribe_callback,
|
db_client_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
db_client_table_done_callback done_callback,
|
db_client_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
db_client_table_subscribe_data *sub_data =
|
DBClientTableSubscribeData *sub_data =
|
||||||
malloc(sizeof(db_client_table_subscribe_data));
|
malloc(sizeof(DBClientTableSubscribeData));
|
||||||
sub_data->subscribe_callback = subscribe_callback;
|
sub_data->subscribe_callback = subscribe_callback;
|
||||||
sub_data->subscribe_context = subscribe_context;
|
sub_data->subscribe_context = subscribe_context;
|
||||||
|
|
||||||
|
|
|
@ -4,7 +4,7 @@
|
||||||
#include "db.h"
|
#include "db.h"
|
||||||
#include "table.h"
|
#include "table.h"
|
||||||
|
|
||||||
typedef void (*db_client_table_done_callback)(db_client_id db_client_id,
|
typedef void (*db_client_table_done_callback)(DBClientID db_client_id,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -12,7 +12,7 @@ typedef void (*db_client_table_done_callback)(db_client_id db_client_id,
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* Callback for subscribing to the db client table. */
|
/* Callback for subscribing to the db client table. */
|
||||||
typedef void (*db_client_table_subscribe_callback)(db_client_id db_client_id,
|
typedef void (*db_client_table_subscribe_callback)(DBClientID db_client_id,
|
||||||
const char *client_type,
|
const char *client_type,
|
||||||
const char *aux_address,
|
const char *aux_address,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
@ -32,10 +32,10 @@ typedef void (*db_client_table_subscribe_callback)(db_client_id db_client_id,
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void db_client_table_subscribe(
|
void db_client_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
db_client_table_subscribe_callback subscribe_callback,
|
db_client_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
db_client_table_done_callback done_callback,
|
db_client_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -44,6 +44,6 @@ void db_client_table_subscribe(
|
||||||
typedef struct {
|
typedef struct {
|
||||||
db_client_table_subscribe_callback subscribe_callback;
|
db_client_table_subscribe_callback subscribe_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} db_client_table_subscribe_data;
|
} DBClientTableSubscribeData;
|
||||||
|
|
||||||
#endif /* DB_CLIENT_TABLE_H */
|
#endif /* DB_CLIENT_TABLE_H */
|
||||||
|
|
|
@ -2,12 +2,12 @@
|
||||||
#include "redis.h"
|
#include "redis.h"
|
||||||
|
|
||||||
void local_scheduler_table_subscribe(
|
void local_scheduler_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
local_scheduler_table_subscribe_callback subscribe_callback,
|
local_scheduler_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry) {
|
RetryInfo *retry) {
|
||||||
local_scheduler_table_subscribe_data *sub_data =
|
LocalSchedulerTableSubscribeData *sub_data =
|
||||||
malloc(sizeof(local_scheduler_table_subscribe_data));
|
malloc(sizeof(LocalSchedulerTableSubscribeData));
|
||||||
sub_data->subscribe_callback = subscribe_callback;
|
sub_data->subscribe_callback = subscribe_callback;
|
||||||
sub_data->subscribe_context = subscribe_context;
|
sub_data->subscribe_context = subscribe_context;
|
||||||
|
|
||||||
|
@ -15,11 +15,11 @@ void local_scheduler_table_subscribe(
|
||||||
redis_local_scheduler_table_subscribe, NULL);
|
redis_local_scheduler_table_subscribe, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
void local_scheduler_table_send_info(db_handle *db_handle,
|
void local_scheduler_table_send_info(DBHandle *db_handle,
|
||||||
local_scheduler_info *info,
|
LocalSchedulerInfo *info,
|
||||||
retry_info *retry) {
|
RetryInfo *retry) {
|
||||||
local_scheduler_table_send_info_data *data =
|
LocalSchedulerTableSendInfoData *data =
|
||||||
malloc(sizeof(local_scheduler_table_send_info_data));
|
malloc(sizeof(LocalSchedulerTableSendInfoData));
|
||||||
data->info = *info;
|
data->info = *info;
|
||||||
|
|
||||||
init_table_callback(db_handle, NIL_ID, __func__, data, retry, NULL,
|
init_table_callback(db_handle, NIL_ID, __func__, data, retry, NULL,
|
||||||
|
|
|
@ -21,7 +21,7 @@ typedef struct {
|
||||||
/** The resource vector of resources currently available to this local
|
/** The resource vector of resources currently available to this local
|
||||||
* scheduler. */
|
* scheduler. */
|
||||||
double dynamic_resources[MAX_RESOURCE_INDEX];
|
double dynamic_resources[MAX_RESOURCE_INDEX];
|
||||||
} local_scheduler_info;
|
} LocalSchedulerInfo;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Subscribing to the local scheduler table ====
|
* ==== Subscribing to the local scheduler table ====
|
||||||
|
@ -29,8 +29,8 @@ typedef struct {
|
||||||
|
|
||||||
/* Callback for subscribing to the local scheduler table. */
|
/* Callback for subscribing to the local scheduler table. */
|
||||||
typedef void (*local_scheduler_table_subscribe_callback)(
|
typedef void (*local_scheduler_table_subscribe_callback)(
|
||||||
db_client_id client_id,
|
DBClientID client_id,
|
||||||
local_scheduler_info info,
|
LocalSchedulerInfo info,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -45,17 +45,17 @@ typedef void (*local_scheduler_table_subscribe_callback)(
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void local_scheduler_table_subscribe(
|
void local_scheduler_table_subscribe(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
local_scheduler_table_subscribe_callback subscribe_callback,
|
local_scheduler_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry);
|
RetryInfo *retry);
|
||||||
|
|
||||||
/* Data that is needed to register local scheduler table subscribe callbacks
|
/* Data that is needed to register local scheduler table subscribe callbacks
|
||||||
* with the state database. */
|
* with the state database. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
local_scheduler_table_subscribe_callback subscribe_callback;
|
local_scheduler_table_subscribe_callback subscribe_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} local_scheduler_table_subscribe_data;
|
} LocalSchedulerTableSubscribeData;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Send a heartbeat to all subscriers to the local scheduler table. This
|
* Send a heartbeat to all subscriers to the local scheduler table. This
|
||||||
|
@ -66,14 +66,15 @@ typedef struct {
|
||||||
* local scheduler.
|
* local scheduler.
|
||||||
* @param retry Information about retrying the request to the database.
|
* @param retry Information about retrying the request to the database.
|
||||||
*/
|
*/
|
||||||
void local_scheduler_table_send_info(db_handle *db_handle,
|
void local_scheduler_table_send_info(DBHandle *db_handle,
|
||||||
local_scheduler_info *info,
|
LocalSchedulerInfo *info,
|
||||||
retry_info *retry);
|
RetryInfo *retry);
|
||||||
|
|
||||||
/* Data that is needed to publish local scheduler heartbeats to the local
|
/* Data that is needed to publish local scheduler heartbeats to the local
|
||||||
* scheduler table. */
|
* scheduler table. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
local_scheduler_info info;
|
/* The information to be sent. */
|
||||||
} local_scheduler_table_send_info_data;
|
LocalSchedulerInfo info;
|
||||||
|
} LocalSchedulerTableSendInfoData;
|
||||||
|
|
||||||
#endif /* LOCAL_SCHEDULER_TABLE_H */
|
#endif /* LOCAL_SCHEDULER_TABLE_H */
|
||||||
|
|
|
@ -2,9 +2,9 @@
|
||||||
#include "redis.h"
|
#include "redis.h"
|
||||||
#include "object_info.h"
|
#include "object_info.h"
|
||||||
|
|
||||||
void object_table_lookup(db_handle *db_handle,
|
void object_table_lookup(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_lookup_done_callback done_callback,
|
object_table_lookup_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(db_handle != NULL);
|
CHECK(db_handle != NULL);
|
||||||
|
@ -12,33 +12,33 @@ void object_table_lookup(db_handle *db_handle,
|
||||||
done_callback, redis_object_table_lookup, user_context);
|
done_callback, redis_object_table_lookup, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_table_add(db_handle *db_handle,
|
void object_table_add(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t object_size,
|
int64_t object_size,
|
||||||
unsigned char digest[],
|
unsigned char digest[],
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_done_callback done_callback,
|
object_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(db_handle != NULL);
|
CHECK(db_handle != NULL);
|
||||||
|
|
||||||
object_table_add_data *info = malloc(sizeof(object_table_add_data));
|
ObjectTableAddData *info = malloc(sizeof(ObjectTableAddData));
|
||||||
info->object_size = object_size;
|
info->object_size = object_size;
|
||||||
memcpy(&info->digest[0], digest, DIGEST_SIZE);
|
memcpy(&info->digest[0], digest, DIGEST_SIZE);
|
||||||
init_table_callback(db_handle, object_id, __func__, info, retry,
|
init_table_callback(db_handle, object_id, __func__, info, retry,
|
||||||
done_callback, redis_object_table_add, user_context);
|
done_callback, redis_object_table_add, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_table_remove(db_handle *db_handle,
|
void object_table_remove(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
db_client_id *client_id,
|
DBClientID *client_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_done_callback done_callback,
|
object_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(db_handle != NULL);
|
CHECK(db_handle != NULL);
|
||||||
/* Copy the client ID, if one was provided. */
|
/* Copy the client ID, if one was provided. */
|
||||||
db_client_id *client_id_copy = NULL;
|
DBClientID *client_id_copy = NULL;
|
||||||
if (client_id != NULL) {
|
if (client_id != NULL) {
|
||||||
client_id_copy = malloc(sizeof(db_client_id));
|
client_id_copy = malloc(sizeof(DBClientID));
|
||||||
*client_id_copy = *client_id;
|
*client_id_copy = *client_id;
|
||||||
}
|
}
|
||||||
init_table_callback(db_handle, object_id, __func__, client_id_copy, retry,
|
init_table_callback(db_handle, object_id, __func__, client_id_copy, retry,
|
||||||
|
@ -46,16 +46,15 @@ void object_table_remove(db_handle *db_handle,
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_table_subscribe_to_notifications(
|
void object_table_subscribe_to_notifications(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
bool subscribe_all,
|
bool subscribe_all,
|
||||||
object_table_object_available_callback object_available_callback,
|
object_table_object_available_callback object_available_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_lookup_done_callback done_callback,
|
object_table_lookup_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(db_handle != NULL);
|
CHECK(db_handle != NULL);
|
||||||
object_table_subscribe_data *sub_data =
|
ObjectTableSubscribeData *sub_data = malloc(sizeof(ObjectTableSubscribeData));
|
||||||
malloc(sizeof(object_table_subscribe_data));
|
|
||||||
sub_data->object_available_callback = object_available_callback;
|
sub_data->object_available_callback = object_available_callback;
|
||||||
sub_data->subscribe_context = subscribe_context;
|
sub_data->subscribe_context = subscribe_context;
|
||||||
sub_data->subscribe_all = subscribe_all;
|
sub_data->subscribe_all = subscribe_all;
|
||||||
|
@ -65,30 +64,29 @@ void object_table_subscribe_to_notifications(
|
||||||
redis_object_table_subscribe_to_notifications, user_context);
|
redis_object_table_subscribe_to_notifications, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_table_request_notifications(db_handle *db_handle,
|
void object_table_request_notifications(DBHandle *db_handle,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
retry_info *retry) {
|
RetryInfo *retry) {
|
||||||
CHECK(db_handle != NULL);
|
CHECK(db_handle != NULL);
|
||||||
CHECK(num_object_ids > 0);
|
CHECK(num_object_ids > 0);
|
||||||
object_table_request_notifications_data *data =
|
ObjectTableRequestNotificationsData *data =
|
||||||
malloc(sizeof(object_table_request_notifications_data) +
|
malloc(sizeof(ObjectTableRequestNotificationsData) +
|
||||||
num_object_ids * sizeof(object_id));
|
num_object_ids * sizeof(ObjectID));
|
||||||
data->num_object_ids = num_object_ids;
|
data->num_object_ids = num_object_ids;
|
||||||
memcpy(data->object_ids, object_ids, num_object_ids * sizeof(object_id));
|
memcpy(data->object_ids, object_ids, num_object_ids * sizeof(ObjectID));
|
||||||
|
|
||||||
init_table_callback(db_handle, NIL_OBJECT_ID, __func__, data, retry, NULL,
|
init_table_callback(db_handle, NIL_OBJECT_ID, __func__, data, retry, NULL,
|
||||||
redis_object_table_request_notifications, NULL);
|
redis_object_table_request_notifications, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_info_subscribe(db_handle *db_handle,
|
void object_info_subscribe(DBHandle *db_handle,
|
||||||
object_info_subscribe_callback subscribe_callback,
|
object_info_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_info_done_callback done_callback,
|
object_info_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
object_info_subscribe_data *sub_data =
|
ObjectInfoSubscribeData *sub_data = malloc(sizeof(ObjectInfoSubscribeData));
|
||||||
malloc(sizeof(object_info_subscribe_data));
|
|
||||||
sub_data->subscribe_callback = subscribe_callback;
|
sub_data->subscribe_callback = subscribe_callback;
|
||||||
sub_data->subscribe_context = subscribe_context;
|
sub_data->subscribe_context = subscribe_context;
|
||||||
|
|
||||||
|
@ -96,21 +94,21 @@ void object_info_subscribe(db_handle *db_handle,
|
||||||
done_callback, redis_object_info_subscribe, user_context);
|
done_callback, redis_object_info_subscribe, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void result_table_add(db_handle *db_handle,
|
void result_table_add(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
task_id task_id_arg,
|
TaskID task_id_arg,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
result_table_done_callback done_callback,
|
result_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
task_id *task_id_copy = malloc(sizeof(task_id));
|
TaskID *task_id_copy = malloc(sizeof(TaskID));
|
||||||
memcpy(task_id_copy, task_id_arg.id, sizeof(*task_id_copy));
|
memcpy(task_id_copy, task_id_arg.id, sizeof(*task_id_copy));
|
||||||
init_table_callback(db_handle, object_id, __func__, task_id_copy, retry,
|
init_table_callback(db_handle, object_id, __func__, task_id_copy, retry,
|
||||||
done_callback, redis_result_table_add, user_context);
|
done_callback, redis_result_table_add, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void result_table_lookup(db_handle *db_handle,
|
void result_table_lookup(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
result_table_lookup_callback done_callback,
|
result_table_lookup_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
init_table_callback(db_handle, object_id, __func__, NULL, retry,
|
init_table_callback(db_handle, object_id, __func__, NULL, retry,
|
||||||
|
|
|
@ -16,14 +16,14 @@
|
||||||
* before), then manager_count will be -1.
|
* before), then manager_count will be -1.
|
||||||
*/
|
*/
|
||||||
typedef void (*object_table_lookup_done_callback)(
|
typedef void (*object_table_lookup_done_callback)(
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
OWNER const char *manager_vector[],
|
OWNER const char *manager_vector[],
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/* Callback called when object object_id is available. */
|
/* Callback called when object ObjectID is available. */
|
||||||
typedef void (*object_table_object_available_callback)(
|
typedef void (*object_table_object_available_callback)(
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
OWNER const char *manager_vector[],
|
OWNER const char *manager_vector[],
|
||||||
|
@ -39,9 +39,9 @@ typedef void (*object_table_object_available_callback)(
|
||||||
* @param user_context Context passed by the caller.
|
* @param user_context Context passed by the caller.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_lookup(db_handle *db_handle,
|
void object_table_lookup(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_lookup_done_callback done_callback,
|
object_table_lookup_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -50,7 +50,7 @@ void object_table_lookup(db_handle *db_handle,
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* Callback called when the object add/remove operation completes. */
|
/* Callback called when the object add/remove operation completes. */
|
||||||
typedef void (*object_table_done_callback)(object_id object_id,
|
typedef void (*object_table_done_callback)(ObjectID object_id,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -65,11 +65,11 @@ typedef void (*object_table_done_callback)(object_id object_id,
|
||||||
* @param user_context User context to be passed in the callbacks.
|
* @param user_context User context to be passed in the callbacks.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_add(db_handle *db_handle,
|
void object_table_add(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t object_size,
|
int64_t object_size,
|
||||||
unsigned char digest[],
|
unsigned char digest[],
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_done_callback done_callback,
|
object_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -77,7 +77,7 @@ void object_table_add(db_handle *db_handle,
|
||||||
typedef struct {
|
typedef struct {
|
||||||
int64_t object_size;
|
int64_t object_size;
|
||||||
unsigned char digest[DIGEST_SIZE];
|
unsigned char digest[DIGEST_SIZE];
|
||||||
} object_table_add_data;
|
} ObjectTableAddData;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Remove object call and callback ====
|
* ==== Remove object call and callback ====
|
||||||
|
@ -96,10 +96,10 @@ typedef struct {
|
||||||
* @param user_context User context to be passed in the callbacks.
|
* @param user_context User context to be passed in the callbacks.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_remove(db_handle *db_handle,
|
void object_table_remove(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
db_client_id *client_id,
|
DBClientID *client_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_done_callback done_callback,
|
object_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -125,11 +125,11 @@ void object_table_remove(db_handle *db_handle,
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_subscribe_to_notifications(
|
void object_table_subscribe_to_notifications(
|
||||||
db_handle *db_handle,
|
DBHandle *db_handle,
|
||||||
bool subscribe_all,
|
bool subscribe_all,
|
||||||
object_table_object_available_callback object_available_callback,
|
object_table_object_available_callback object_available_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_table_lookup_done_callback done_callback,
|
object_table_lookup_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -144,18 +144,18 @@ void object_table_subscribe_to_notifications(
|
||||||
* @param retry Information about retrying the request to the database.
|
* @param retry Information about retrying the request to the database.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_request_notifications(db_handle *db,
|
void object_table_request_notifications(DBHandle *db,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
retry_info *retry);
|
RetryInfo *retry);
|
||||||
|
|
||||||
/** Data that is needed to run object_request_notifications requests. */
|
/** Data that is needed to run object_request_notifications requests. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The number of object IDs. */
|
/** The number of object IDs. */
|
||||||
int num_object_ids;
|
int num_object_ids;
|
||||||
/** This field is used to store a variable number of object IDs. */
|
/** This field is used to store a variable number of object IDs. */
|
||||||
object_id object_ids[0];
|
ObjectID object_ids[0];
|
||||||
} object_table_request_notifications_data;
|
} ObjectTableRequestNotificationsData;
|
||||||
|
|
||||||
/** Data that is needed to register new object available callbacks with the
|
/** Data that is needed to register new object available callbacks with the
|
||||||
* state database. */
|
* state database. */
|
||||||
|
@ -163,16 +163,16 @@ typedef struct {
|
||||||
bool subscribe_all;
|
bool subscribe_all;
|
||||||
object_table_object_available_callback object_available_callback;
|
object_table_object_available_callback object_available_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} object_table_subscribe_data;
|
} ObjectTableSubscribeData;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Object info table, contains size of the object ====
|
* ==== Object info table, contains size of the object ====
|
||||||
*/
|
*/
|
||||||
|
|
||||||
typedef void (*object_info_done_callback)(object_id object_id,
|
typedef void (*object_info_done_callback)(ObjectID object_id,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
typedef void (*object_info_subscribe_callback)(object_id object_id,
|
typedef void (*object_info_subscribe_callback)(ObjectID object_id,
|
||||||
int64_t object_size,
|
int64_t object_size,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -190,10 +190,10 @@ typedef void (*object_info_subscribe_callback)(object_id object_id,
|
||||||
* callbacks.
|
* callbacks.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_info_subscribe(db_handle *db_handle,
|
void object_info_subscribe(DBHandle *db_handle,
|
||||||
object_info_subscribe_callback subscribe_callback,
|
object_info_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
object_info_done_callback done_callback,
|
object_info_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -202,7 +202,7 @@ void object_info_subscribe(db_handle *db_handle,
|
||||||
typedef struct {
|
typedef struct {
|
||||||
object_info_subscribe_callback subscribe_callback;
|
object_info_subscribe_callback subscribe_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} object_info_subscribe_data;
|
} ObjectInfoSubscribeData;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Result table ====
|
* ==== Result table ====
|
||||||
|
@ -211,7 +211,7 @@ typedef struct {
|
||||||
/**
|
/**
|
||||||
* Callback called when the add/remove operation for a result table entry
|
* Callback called when the add/remove operation for a result table entry
|
||||||
* completes. */
|
* completes. */
|
||||||
typedef void (*result_table_done_callback)(object_id object_id,
|
typedef void (*result_table_done_callback)(ObjectID object_id,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -227,16 +227,16 @@ typedef void (*result_table_done_callback)(object_id object_id,
|
||||||
* @param user_context Context passed by the caller.
|
* @param user_context Context passed by the caller.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void result_table_add(db_handle *db_handle,
|
void result_table_add(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
result_table_done_callback done_callback,
|
result_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/** Callback called when the result table lookup completes. */
|
/** Callback called when the result table lookup completes. */
|
||||||
typedef void (*result_table_lookup_callback)(object_id object_id,
|
typedef void (*result_table_lookup_callback)(ObjectID object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -250,9 +250,9 @@ typedef void (*result_table_lookup_callback)(object_id object_id,
|
||||||
* @param user_context Context passed by the caller.
|
* @param user_context Context passed by the caller.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void result_table_lookup(db_handle *db_handle,
|
void result_table_lookup(DBHandle *db_handle,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
result_table_lookup_callback done_callback,
|
result_table_lookup_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
|
|
@ -54,54 +54,33 @@ extern int usleep(useconds_t usec);
|
||||||
* already removed, meaning that the operation was already marked as succeeded
|
* already removed, meaning that the operation was already marked as succeeded
|
||||||
* or failed.
|
* or failed.
|
||||||
*/
|
*/
|
||||||
#define REDIS_CALLBACK_HEADER(DB, CB_DATA, REPLY) \
|
#define REDIS_CALLBACK_HEADER(DB, CB_DATA, REPLY) \
|
||||||
if ((REPLY) == NULL) { \
|
if ((REPLY) == NULL) { \
|
||||||
return; \
|
return; \
|
||||||
} \
|
} \
|
||||||
db_handle *DB = c->data; \
|
DBHandle *DB = (DBHandle *) c->data; \
|
||||||
table_callback_data *CB_DATA = \
|
TableCallbackData *CB_DATA = outstanding_callbacks_find((int64_t) privdata); \
|
||||||
outstanding_callbacks_find((int64_t) privdata); \
|
if (CB_DATA == NULL) { \
|
||||||
if (CB_DATA == NULL) { \
|
/* the callback data structure has been \
|
||||||
/* the callback data structure has been \
|
* already freed; just ignore this reply */ \
|
||||||
* already freed; just ignore this reply */ \
|
return; \
|
||||||
return; \
|
} \
|
||||||
} \
|
do { \
|
||||||
do { \
|
|
||||||
} while (0)
|
} while (0)
|
||||||
|
|
||||||
/**
|
DBHandle *db_connect(const char *db_address,
|
||||||
* A data structure to track the status of a table operation attempt that spans
|
int db_port,
|
||||||
* multiple Redis commands. Each attempt at a table operation is associated
|
const char *client_type,
|
||||||
* with a unique redis_requests_info instance. To use this data structure, pass
|
const char *node_ip_address,
|
||||||
* it as the `privdata` argument for the callback of each asynchronous Redis
|
int num_args,
|
||||||
* command.
|
const char **args) {
|
||||||
*/
|
|
||||||
typedef struct {
|
|
||||||
/** The timer ID that uniquely identifies this table operation. All retry
|
|
||||||
* attempts of a table operation share the same timer ID. */
|
|
||||||
int64_t timer_id;
|
|
||||||
/** The index of the next command to try for this operation. This may be
|
|
||||||
* different across different attempts of the same table operation. */
|
|
||||||
int request_index;
|
|
||||||
/** Whether the current invocation of the callback was triggered by a reply
|
|
||||||
* to an asynchronous Redis command. If not, then the callback was called
|
|
||||||
* directly. */
|
|
||||||
bool is_redis_reply;
|
|
||||||
} redis_requests_info;
|
|
||||||
|
|
||||||
db_handle *db_connect(const char *db_address,
|
|
||||||
int db_port,
|
|
||||||
const char *client_type,
|
|
||||||
const char *node_ip_address,
|
|
||||||
int num_args,
|
|
||||||
const char **args) {
|
|
||||||
/* Check that the number of args is even. These args will be passed to the
|
/* Check that the number of args is even. These args will be passed to the
|
||||||
* RAY.CONNECT Redis command, which takes arguments in pairs. */
|
* RAY.CONNECT Redis command, which takes arguments in pairs. */
|
||||||
if (num_args % 2 != 0) {
|
if (num_args % 2 != 0) {
|
||||||
LOG_FATAL("The number of extra args must be divisible by two.");
|
LOG_FATAL("The number of extra args must be divisible by two.");
|
||||||
}
|
}
|
||||||
|
|
||||||
db_handle *db = malloc(sizeof(db_handle));
|
DBHandle *db = (DBHandle *) malloc(sizeof(DBHandle));
|
||||||
/* Sync connection for initial handshake */
|
/* Sync connection for initial handshake */
|
||||||
redisReply *reply;
|
redisReply *reply;
|
||||||
int connection_attempts = 0;
|
int connection_attempts = 0;
|
||||||
|
@ -124,21 +103,22 @@ db_handle *db_connect(const char *db_address,
|
||||||
* should only need to be done once (by whoever started Redis), but since
|
* should only need to be done once (by whoever started Redis), but since
|
||||||
* Redis may be started in multiple places (e.g., for testing or when starting
|
* Redis may be started in multiple places (e.g., for testing or when starting
|
||||||
* processes by hand), it is easier to do it multiple times. */
|
* processes by hand), it is easier to do it multiple times. */
|
||||||
reply = redisCommand(context, "CONFIG SET notify-keyspace-events Kl");
|
reply = (redisReply *) redisCommand(context,
|
||||||
|
"CONFIG SET notify-keyspace-events Kl");
|
||||||
CHECKM(reply != NULL, "db_connect failed on CONFIG SET");
|
CHECKM(reply != NULL, "db_connect failed on CONFIG SET");
|
||||||
freeReplyObject(reply);
|
freeReplyObject(reply);
|
||||||
/* Also configure Redis to not run in protected mode, so clients on other
|
/* Also configure Redis to not run in protected mode, so clients on other
|
||||||
* hosts can connect to it. */
|
* hosts can connect to it. */
|
||||||
reply = redisCommand(context, "CONFIG SET protected-mode no");
|
reply = (redisReply *) redisCommand(context, "CONFIG SET protected-mode no");
|
||||||
CHECKM(reply != NULL, "db_connect failed on CONFIG SET");
|
CHECKM(reply != NULL, "db_connect failed on CONFIG SET");
|
||||||
freeReplyObject(reply);
|
freeReplyObject(reply);
|
||||||
/* Create a client ID for this client. */
|
/* Create a client ID for this client. */
|
||||||
db_client_id client = globally_unique_id();
|
DBClientID client = globally_unique_id();
|
||||||
|
|
||||||
/* Construct the argument arrays for RAY.CONNECT. */
|
/* Construct the argument arrays for RAY.CONNECT. */
|
||||||
int argc = num_args + 4;
|
int argc = num_args + 4;
|
||||||
const char **argv = malloc(sizeof(char *) * argc);
|
const char **argv = (const char **) malloc(sizeof(char *) * argc);
|
||||||
size_t *argvlen = malloc(sizeof(size_t) * argc);
|
size_t *argvlen = (size_t *) malloc(sizeof(size_t) * argc);
|
||||||
/* Set the command name argument. */
|
/* Set the command name argument. */
|
||||||
argv[0] = "RAY.CONNECT";
|
argv[0] = "RAY.CONNECT";
|
||||||
argvlen[0] = strlen(argv[0]);
|
argvlen[0] = strlen(argv[0]);
|
||||||
|
@ -163,7 +143,7 @@ db_handle *db_connect(const char *db_address,
|
||||||
|
|
||||||
/* Register this client with Redis. RAY.CONNECT is a custom Redis command that
|
/* Register this client with Redis. RAY.CONNECT is a custom Redis command that
|
||||||
* we've defined. */
|
* we've defined. */
|
||||||
reply = redisCommandArgv(context, argc, argv, argvlen);
|
reply = (redisReply *) redisCommandArgv(context, argc, argv, argvlen);
|
||||||
CHECKM(reply != NULL, "db_connect failed on RAY.CONNECT");
|
CHECKM(reply != NULL, "db_connect failed on RAY.CONNECT");
|
||||||
CHECK(reply->type != REDIS_REPLY_ERROR);
|
CHECK(reply->type != REDIS_REPLY_ERROR);
|
||||||
CHECK(strcmp(reply->str, "OK") == 0);
|
CHECK(strcmp(reply->str, "OK") == 0);
|
||||||
|
@ -194,11 +174,11 @@ db_handle *db_connect(const char *db_address,
|
||||||
return db;
|
return db;
|
||||||
}
|
}
|
||||||
|
|
||||||
void db_disconnect(db_handle *db) {
|
void db_disconnect(DBHandle *db) {
|
||||||
redisFree(db->sync_context);
|
redisFree(db->sync_context);
|
||||||
redisAsyncFree(db->context);
|
redisAsyncFree(db->context);
|
||||||
redisAsyncFree(db->sub_context);
|
redisAsyncFree(db->sub_context);
|
||||||
db_client_cache_entry *e, *tmp;
|
DBClientCacheEntry *e, *tmp;
|
||||||
HASH_ITER(hh, db->db_client_cache, e, tmp) {
|
HASH_ITER(hh, db->db_client_cache, e, tmp) {
|
||||||
free(e->addr);
|
free(e->addr);
|
||||||
HASH_DELETE(hh, db->db_client_cache, e);
|
HASH_DELETE(hh, db->db_client_cache, e);
|
||||||
|
@ -208,7 +188,7 @@ void db_disconnect(db_handle *db) {
|
||||||
free(db);
|
free(db);
|
||||||
}
|
}
|
||||||
|
|
||||||
void db_attach(db_handle *db, event_loop *loop, bool reattach) {
|
void db_attach(DBHandle *db, event_loop *loop, bool reattach) {
|
||||||
db->loop = loop;
|
db->loop = loop;
|
||||||
int err = redisAeAttach(loop, db->context);
|
int err = redisAeAttach(loop, db->context);
|
||||||
/* If the database is reattached in the tests, redis normally gives
|
/* If the database is reattached in the tests, redis normally gives
|
||||||
|
@ -232,7 +212,7 @@ void redis_object_table_add_callback(redisAsyncContext *c,
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
/* Do some minimal checking. */
|
/* Do some minimal checking. */
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
if (strcmp(reply->str, "hash mismatch") == 0) {
|
if (strcmp(reply->str, "hash mismatch") == 0) {
|
||||||
/* If our object hash doesn't match the one recorded in the table, report
|
/* If our object hash doesn't match the one recorded in the table, report
|
||||||
* the error back to the user and exit immediately. */
|
* the error back to the user and exit immediately. */
|
||||||
|
@ -245,18 +225,19 @@ void redis_object_table_add_callback(redisAsyncContext *c,
|
||||||
CHECK(strcmp(reply->str, "OK") == 0);
|
CHECK(strcmp(reply->str, "OK") == 0);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
object_table_done_callback done_callback = callback_data->done_callback;
|
object_table_done_callback done_callback =
|
||||||
|
(object_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_table_add(table_callback_data *callback_data) {
|
void redis_object_table_add(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
|
|
||||||
object_table_add_data *info = callback_data->data;
|
ObjectTableAddData *info = (ObjectTableAddData *) callback_data->data;
|
||||||
object_id obj_id = callback_data->id;
|
ObjectID obj_id = callback_data->id;
|
||||||
int64_t object_size = info->object_size;
|
int64_t object_size = info->object_size;
|
||||||
unsigned char *digest = info->digest;
|
unsigned char *digest = info->digest;
|
||||||
|
|
||||||
|
@ -277,7 +258,7 @@ void redis_object_table_remove_callback(redisAsyncContext *c,
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
/* Do some minimal checking. */
|
/* Do some minimal checking. */
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
if (strcmp(reply->str, "object not found") == 0) {
|
if (strcmp(reply->str, "object not found") == 0) {
|
||||||
/* If our object entry was not in the table, it's probably a race
|
/* If our object entry was not in the table, it's probably a race
|
||||||
* condition with an object_table_add. */
|
* condition with an object_table_add. */
|
||||||
|
@ -287,20 +268,21 @@ void redis_object_table_remove_callback(redisAsyncContext *c,
|
||||||
CHECK(strcmp(reply->str, "OK") == 0);
|
CHECK(strcmp(reply->str, "OK") == 0);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
object_table_done_callback done_callback = callback_data->done_callback;
|
object_table_done_callback done_callback =
|
||||||
|
(object_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_table_remove(table_callback_data *callback_data) {
|
void redis_object_table_remove(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
|
|
||||||
object_id obj_id = callback_data->id;
|
ObjectID obj_id = callback_data->id;
|
||||||
/* If the caller provided a manager ID to delete, use it. Otherwise, use our
|
/* If the caller provided a manager ID to delete, use it. Otherwise, use our
|
||||||
* own client ID as the ID to delete. */
|
* own client ID as the ID to delete. */
|
||||||
db_client_id *client_id = callback_data->data;
|
DBClientID *client_id = (DBClientID *) callback_data->data;
|
||||||
if (client_id == NULL) {
|
if (client_id == NULL) {
|
||||||
client_id = &db->client;
|
client_id = &db->client;
|
||||||
}
|
}
|
||||||
|
@ -314,11 +296,11 @@ void redis_object_table_remove(table_callback_data *callback_data) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_table_lookup(table_callback_data *callback_data) {
|
void redis_object_table_lookup(TableCallbackData *callback_data) {
|
||||||
CHECK(callback_data);
|
CHECK(callback_data);
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
|
|
||||||
object_id obj_id = callback_data->id;
|
ObjectID obj_id = callback_data->id;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, redis_object_table_lookup_callback,
|
db->context, redis_object_table_lookup_callback,
|
||||||
(void *) callback_data->timer_id, "RAY.OBJECT_TABLE_LOOKUP %b", obj_id.id,
|
(void *) callback_data->timer_id, "RAY.OBJECT_TABLE_LOOKUP %b", obj_id.id,
|
||||||
|
@ -332,24 +314,25 @@ void redis_result_table_add_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
/* Check that the command succeeded. */
|
/* Check that the command succeeded. */
|
||||||
CHECK(reply->type != REDIS_REPLY_ERROR);
|
CHECK(reply->type != REDIS_REPLY_ERROR);
|
||||||
CHECKM(strncmp(reply->str, "OK", strlen("OK")) == 0, "reply->str is %s",
|
CHECKM(strncmp(reply->str, "OK", strlen("OK")) == 0, "reply->str is %s",
|
||||||
reply->str);
|
reply->str);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
if (callback_data->done_callback) {
|
if (callback_data->done_callback) {
|
||||||
result_table_done_callback done_callback = callback_data->done_callback;
|
result_table_done_callback done_callback =
|
||||||
|
(result_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_result_table_add(table_callback_data *callback_data) {
|
void redis_result_table_add(TableCallbackData *callback_data) {
|
||||||
CHECK(callback_data);
|
CHECK(callback_data);
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
object_id id = callback_data->id;
|
ObjectID id = callback_data->id;
|
||||||
task_id *result_task_id = (task_id *) callback_data->data;
|
TaskID *result_task_id = (TaskID *) callback_data->data;
|
||||||
/* Add the result entry to the result table. */
|
/* Add the result entry to the result table. */
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, redis_result_table_add_callback,
|
db->context, redis_result_table_add_callback,
|
||||||
|
@ -363,8 +346,8 @@ void redis_result_table_add(table_callback_data *callback_data) {
|
||||||
/* This allocates a task which must be freed by the caller, unless the returned
|
/* This allocates a task which must be freed by the caller, unless the returned
|
||||||
* task is NULL. This is used by both redis_result_table_lookup_callback and
|
* task is NULL. This is used by both redis_result_table_lookup_callback and
|
||||||
* redis_task_table_get_task_callback. */
|
* redis_task_table_get_task_callback. */
|
||||||
task *parse_and_construct_task_from_redis_reply(redisReply *reply) {
|
Task *parse_and_construct_task_from_redis_reply(redisReply *reply) {
|
||||||
task *task;
|
Task *task;
|
||||||
if (reply->type == REDIS_REPLY_NIL) {
|
if (reply->type == REDIS_REPLY_NIL) {
|
||||||
/* There is no task in the reply, so return NULL. */
|
/* There is no task in the reply, so return NULL. */
|
||||||
task = NULL;
|
task = NULL;
|
||||||
|
@ -379,15 +362,15 @@ task *parse_and_construct_task_from_redis_reply(redisReply *reply) {
|
||||||
/* Parse the scheduling state. */
|
/* Parse the scheduling state. */
|
||||||
long long state = reply->element[0]->integer;
|
long long state = reply->element[0]->integer;
|
||||||
/* Parse the local scheduler db_client_id. */
|
/* Parse the local scheduler db_client_id. */
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
CHECK(sizeof(local_scheduler_id) == reply->element[1]->len);
|
CHECK(sizeof(local_scheduler_id) == reply->element[1]->len);
|
||||||
memcpy(local_scheduler_id.id, reply->element[1]->str,
|
memcpy(local_scheduler_id.id, reply->element[1]->str,
|
||||||
reply->element[1]->len);
|
reply->element[1]->len);
|
||||||
/* Parse the task spec. */
|
/* Parse the task spec. */
|
||||||
task_spec *spec = malloc(reply->element[2]->len);
|
task_spec *spec = (task_spec *) malloc(reply->element[2]->len);
|
||||||
memcpy(spec, reply->element[2]->str, reply->element[2]->len);
|
memcpy(spec, reply->element[2]->str, reply->element[2]->len);
|
||||||
CHECK(task_spec_size(spec) == reply->element[2]->len);
|
CHECK(task_spec_size(spec) == reply->element[2]->len);
|
||||||
task = alloc_task(spec, state, local_scheduler_id);
|
task = Task_alloc(spec, state, local_scheduler_id);
|
||||||
/* Free the task spec. */
|
/* Free the task spec. */
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
} else {
|
} else {
|
||||||
|
@ -401,19 +384,20 @@ void redis_result_table_lookup_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECKM(reply->type == REDIS_REPLY_NIL || reply->type == REDIS_REPLY_STRING,
|
CHECKM(reply->type == REDIS_REPLY_NIL || reply->type == REDIS_REPLY_STRING,
|
||||||
"Unexpected reply type %d in redis_result_table_lookup_callback",
|
"Unexpected reply type %d in redis_result_table_lookup_callback",
|
||||||
reply->type);
|
reply->type);
|
||||||
/* Parse the task from the reply. */
|
/* Parse the task from the reply. */
|
||||||
task_id result_id = NIL_TASK_ID;
|
TaskID result_id = NIL_TASK_ID;
|
||||||
if (reply->type == REDIS_REPLY_STRING) {
|
if (reply->type == REDIS_REPLY_STRING) {
|
||||||
CHECK(reply->len == sizeof(result_id));
|
CHECK(reply->len == sizeof(result_id));
|
||||||
memcpy(&result_id, reply->str, reply->len);
|
memcpy(&result_id, reply->str, reply->len);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
result_table_lookup_callback done_callback = callback_data->done_callback;
|
result_table_lookup_callback done_callback =
|
||||||
|
(result_table_lookup_callback) callback_data->done_callback;
|
||||||
if (done_callback != NULL) {
|
if (done_callback != NULL) {
|
||||||
done_callback(callback_data->id, result_id, callback_data->user_context);
|
done_callback(callback_data->id, result_id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
|
@ -421,10 +405,10 @@ void redis_result_table_lookup_callback(redisAsyncContext *c,
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_result_table_lookup(table_callback_data *callback_data) {
|
void redis_result_table_lookup(TableCallbackData *callback_data) {
|
||||||
CHECK(callback_data);
|
CHECK(callback_data);
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
object_id id = callback_data->id;
|
ObjectID id = callback_data->id;
|
||||||
int status =
|
int status =
|
||||||
redisAsyncCommand(db->context, redis_result_table_lookup_callback,
|
redisAsyncCommand(db->context, redis_result_table_lookup_callback,
|
||||||
(void *) callback_data->timer_id,
|
(void *) callback_data->timer_id,
|
||||||
|
@ -442,20 +426,20 @@ void redis_result_table_lookup(table_callback_data *callback_data) {
|
||||||
* @param manager The pointer where the IP address of the manager gets written.
|
* @param manager The pointer where the IP address of the manager gets written.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_get_cached_db_client(db_handle *db,
|
void redis_get_cached_db_client(DBHandle *db,
|
||||||
db_client_id db_client_id,
|
DBClientID db_client_id,
|
||||||
const char **manager) {
|
const char **manager) {
|
||||||
db_client_cache_entry *entry;
|
DBClientCacheEntry *entry;
|
||||||
HASH_FIND(hh, db->db_client_cache, &db_client_id, sizeof(db_client_id),
|
HASH_FIND(hh, db->db_client_cache, &db_client_id, sizeof(db_client_id),
|
||||||
entry);
|
entry);
|
||||||
if (!entry) {
|
if (!entry) {
|
||||||
/* This is a very rare case. It should happen at most once per db client. */
|
/* This is a very rare case. It should happen at most once per db client. */
|
||||||
redisReply *reply =
|
redisReply *reply = (redisReply *) redisCommand(
|
||||||
redisCommand(db->sync_context, "RAY.GET_CLIENT_ADDRESS %b",
|
db->sync_context, "RAY.GET_CLIENT_ADDRESS %b", (char *) db_client_id.id,
|
||||||
(char *) db_client_id.id, sizeof(db_client_id.id));
|
sizeof(db_client_id.id));
|
||||||
CHECKM(reply->type == REDIS_REPLY_STRING, "REDIS reply type=%d, str=%s",
|
CHECKM(reply->type == REDIS_REPLY_STRING, "REDIS reply type=%d, str=%s",
|
||||||
reply->type, reply->str);
|
reply->type, reply->str);
|
||||||
entry = malloc(sizeof(db_client_cache_entry));
|
entry = (DBClientCacheEntry *) malloc(sizeof(DBClientCacheEntry));
|
||||||
entry->db_client_id = db_client_id;
|
entry->db_client_id = db_client_id;
|
||||||
entry->addr = strdup(reply->str);
|
entry->addr = strdup(reply->str);
|
||||||
HASH_ADD(hh, db->db_client_cache, db_client_id, sizeof(db_client_id),
|
HASH_ADD(hh, db->db_client_cache, db_client_id, sizeof(db_client_id),
|
||||||
|
@ -469,13 +453,13 @@ void redis_object_table_lookup_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
LOG_DEBUG("Object table lookup callback");
|
LOG_DEBUG("Object table lookup callback");
|
||||||
CHECK(reply->type == REDIS_REPLY_NIL || reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_NIL || reply->type == REDIS_REPLY_ARRAY);
|
||||||
|
|
||||||
object_id obj_id = callback_data->id;
|
ObjectID obj_id = callback_data->id;
|
||||||
int64_t manager_count = 0;
|
int64_t manager_count = 0;
|
||||||
db_client_id *managers = NULL;
|
DBClientID *managers = NULL;
|
||||||
const char **manager_vector = NULL;
|
const char **manager_vector = NULL;
|
||||||
|
|
||||||
/* Parse the Redis reply. */
|
/* Parse the Redis reply. */
|
||||||
|
@ -485,8 +469,8 @@ void redis_object_table_lookup_callback(redisAsyncContext *c,
|
||||||
} else if (reply->type == REDIS_REPLY_ARRAY) {
|
} else if (reply->type == REDIS_REPLY_ARRAY) {
|
||||||
manager_count = reply->elements;
|
manager_count = reply->elements;
|
||||||
if (manager_count > 0) {
|
if (manager_count > 0) {
|
||||||
managers = malloc(reply->elements * sizeof(db_client_id));
|
managers = (DBClientID *) malloc(reply->elements * sizeof(DBClientID));
|
||||||
manager_vector = malloc(manager_count * sizeof(char *));
|
manager_vector = (const char **) malloc(manager_count * sizeof(char *));
|
||||||
}
|
}
|
||||||
for (int j = 0; j < reply->elements; ++j) {
|
for (int j = 0; j < reply->elements; ++j) {
|
||||||
CHECK(reply->element[j]->type == REDIS_REPLY_STRING);
|
CHECK(reply->element[j]->type == REDIS_REPLY_STRING);
|
||||||
|
@ -498,7 +482,7 @@ void redis_object_table_lookup_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
|
|
||||||
object_table_lookup_done_callback done_callback =
|
object_table_lookup_done_callback done_callback =
|
||||||
callback_data->done_callback;
|
(object_table_lookup_done_callback) callback_data->done_callback;
|
||||||
if (done_callback) {
|
if (done_callback) {
|
||||||
done_callback(obj_id, manager_count, manager_vector,
|
done_callback(obj_id, manager_count, manager_vector,
|
||||||
callback_data->user_context);
|
callback_data->user_context);
|
||||||
|
@ -530,50 +514,50 @@ void redis_object_table_lookup_callback(redisAsyncContext *c,
|
||||||
* The caller is responsible for freeing this array.
|
* The caller is responsible for freeing this array.
|
||||||
* @return The object ID that the notification is about.
|
* @return The object ID that the notification is about.
|
||||||
*/
|
*/
|
||||||
object_id parse_subscribe_to_notifications_payload(
|
ObjectID parse_subscribe_to_notifications_payload(
|
||||||
db_handle *db,
|
DBHandle *db,
|
||||||
char *payload,
|
char *payload,
|
||||||
int length,
|
int length,
|
||||||
int64_t *data_size,
|
int64_t *data_size,
|
||||||
int *manager_count,
|
int *manager_count,
|
||||||
const char ***manager_vector) {
|
const char ***manager_vector) {
|
||||||
long long data_size_value = 0;
|
long long data_size_value = 0;
|
||||||
int num_managers = (length - sizeof(object_id) - 1 - sizeof(data_size_value) -
|
int num_managers = (length - sizeof(ObjectID) - 1 - sizeof(data_size_value) -
|
||||||
1 - strlen("MANAGERS")) /
|
1 - strlen("MANAGERS")) /
|
||||||
(1 + sizeof(db_client_id));
|
(1 + sizeof(DBClientID));
|
||||||
|
|
||||||
int64_t rval = sizeof(object_id) + 1 + sizeof(data_size_value) + 1 +
|
int64_t rval = sizeof(ObjectID) + 1 + sizeof(data_size_value) + 1 +
|
||||||
strlen("MANAGERS") + num_managers * (1 + sizeof(db_client_id));
|
strlen("MANAGERS") + num_managers * (1 + sizeof(DBClientID));
|
||||||
|
|
||||||
CHECKM(length == rval,
|
CHECKM(length == rval,
|
||||||
"length mismatch: num_managers = %d, length = %d, rval = %" PRId64,
|
"length mismatch: num_managers = %d, length = %d, rval = %" PRId64,
|
||||||
num_managers, length, rval);
|
num_managers, length, rval);
|
||||||
CHECK(num_managers > 0);
|
CHECK(num_managers > 0);
|
||||||
object_id obj_id;
|
ObjectID obj_id;
|
||||||
/* Track our current offset in the payload. */
|
/* Track our current offset in the payload. */
|
||||||
int offset = 0;
|
int offset = 0;
|
||||||
/* Parse the object ID. */
|
/* Parse the object ID. */
|
||||||
memcpy(&obj_id.id, &payload[offset], sizeof(obj_id.id));
|
memcpy(&obj_id.id, &payload[offset], sizeof(obj_id.id));
|
||||||
offset += sizeof(obj_id.id);
|
offset += sizeof(obj_id.id);
|
||||||
/* The next part of the payload is a space. */
|
/* The next part of the payload is a space. */
|
||||||
char *space_str = " ";
|
const char *space_str = " ";
|
||||||
CHECK(memcmp(&payload[offset], space_str, strlen(space_str)) == 0);
|
CHECK(memcmp(&payload[offset], space_str, strlen(space_str)) == 0);
|
||||||
offset += strlen(space_str);
|
offset += strlen(space_str);
|
||||||
/* The next part of the payload is binary data_size. */
|
/* The next part of the payload is binary data_size. */
|
||||||
memcpy(&data_size_value, &payload[offset], sizeof(data_size_value));
|
memcpy(&data_size_value, &payload[offset], sizeof(data_size_value));
|
||||||
offset += sizeof(data_size_value);
|
offset += sizeof(data_size_value);
|
||||||
/* The next part of the payload is the string " MANAGERS" with leading ' '. */
|
/* The next part of the payload is the string " MANAGERS" with leading ' '. */
|
||||||
char *managers_str = " MANAGERS";
|
const char *managers_str = " MANAGERS";
|
||||||
CHECK(memcmp(&payload[offset], managers_str, strlen(managers_str)) == 0);
|
CHECK(memcmp(&payload[offset], managers_str, strlen(managers_str)) == 0);
|
||||||
offset += strlen(managers_str);
|
offset += strlen(managers_str);
|
||||||
/* Parse the managers. */
|
/* Parse the managers. */
|
||||||
const char **managers = malloc(num_managers * sizeof(char *));
|
const char **managers = (const char **) malloc(num_managers * sizeof(char *));
|
||||||
for (int i = 0; i < num_managers; ++i) {
|
for (int i = 0; i < num_managers; ++i) {
|
||||||
/* First there is a space. */
|
/* First there is a space. */
|
||||||
CHECK(memcmp(&payload[offset], " ", strlen(" ")) == 0);
|
CHECK(memcmp(&payload[offset], " ", strlen(" ")) == 0);
|
||||||
offset += strlen(" ");
|
offset += strlen(" ");
|
||||||
/* Get the manager ID. */
|
/* Get the manager ID. */
|
||||||
db_client_id manager_id;
|
DBClientID manager_id;
|
||||||
memcpy(&manager_id.id, &payload[offset], sizeof(manager_id.id));
|
memcpy(&manager_id.id, &payload[offset], sizeof(manager_id.id));
|
||||||
offset += sizeof(manager_id.id);
|
offset += sizeof(manager_id.id);
|
||||||
/* Write the address of the corresponding manager to the returned array. */
|
/* Write the address of the corresponding manager to the returned array. */
|
||||||
|
@ -605,7 +589,7 @@ void object_table_redis_subscribe_to_notifications_callback(
|
||||||
* - reply->element[1]->str is the name of the channel
|
* - reply->element[1]->str is the name of the channel
|
||||||
* - reply->emement[2]->str is the contents of the message.
|
* - reply->emement[2]->str is the contents of the message.
|
||||||
*/
|
*/
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
CHECK(reply->elements == 3);
|
CHECK(reply->elements == 3);
|
||||||
redisReply *message_type = reply->element[0];
|
redisReply *message_type = reply->element[0];
|
||||||
|
@ -617,11 +601,12 @@ void object_table_redis_subscribe_to_notifications_callback(
|
||||||
int64_t data_size = 0;
|
int64_t data_size = 0;
|
||||||
int manager_count;
|
int manager_count;
|
||||||
const char **manager_vector;
|
const char **manager_vector;
|
||||||
object_id obj_id = parse_subscribe_to_notifications_payload(
|
ObjectID obj_id = parse_subscribe_to_notifications_payload(
|
||||||
db, reply->element[2]->str, reply->element[2]->len, &data_size,
|
db, reply->element[2]->str, reply->element[2]->len, &data_size,
|
||||||
&manager_count, &manager_vector);
|
&manager_count, &manager_vector);
|
||||||
/* Call the subscribe callback. */
|
/* Call the subscribe callback. */
|
||||||
object_table_subscribe_data *data = callback_data->data;
|
ObjectTableSubscribeData *data =
|
||||||
|
(ObjectTableSubscribeData *) callback_data->data;
|
||||||
if (data->object_available_callback) {
|
if (data->object_available_callback) {
|
||||||
data->object_available_callback(obj_id, data_size, manager_count,
|
data->object_available_callback(obj_id, data_size, manager_count,
|
||||||
manager_vector, data->subscribe_context);
|
manager_vector, data->subscribe_context);
|
||||||
|
@ -633,7 +618,7 @@ void object_table_redis_subscribe_to_notifications_callback(
|
||||||
* used in the tests. */
|
* used in the tests. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
object_table_lookup_done_callback done_callback =
|
object_table_lookup_done_callback done_callback =
|
||||||
callback_data->done_callback;
|
(object_table_lookup_done_callback) callback_data->done_callback;
|
||||||
done_callback(NIL_ID, 0, NULL, callback_data->user_context);
|
done_callback(NIL_ID, 0, NULL, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* If the initial SUBSCRIBE was successful, clean up the timer, but don't
|
/* If the initial SUBSCRIBE was successful, clean up the timer, but don't
|
||||||
|
@ -647,8 +632,8 @@ void object_table_redis_subscribe_to_notifications_callback(
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_table_subscribe_to_notifications(
|
void redis_object_table_subscribe_to_notifications(
|
||||||
table_callback_data *callback_data) {
|
TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
/* The object channel prefix must match the value defined in
|
/* The object channel prefix must match the value defined in
|
||||||
* src/common/redismodule/ray_redis_module.c. */
|
* src/common/redismodule/ray_redis_module.c. */
|
||||||
const char *object_channel_prefix = "OC:";
|
const char *object_channel_prefix = "OC:";
|
||||||
|
@ -659,7 +644,7 @@ void redis_object_table_subscribe_to_notifications(
|
||||||
* The channel name should probably be the client ID with some prefix. */
|
* The channel name should probably be the client ID with some prefix. */
|
||||||
CHECKM(callback_data->data != NULL,
|
CHECKM(callback_data->data != NULL,
|
||||||
"Object table subscribe data passed as NULL.");
|
"Object table subscribe data passed as NULL.");
|
||||||
if (((object_table_subscribe_data *) (callback_data->data))->subscribe_all) {
|
if (((ObjectTableSubscribeData *) (callback_data->data))->subscribe_all) {
|
||||||
/* Subscribe to the object broadcast channel. */
|
/* Subscribe to the object broadcast channel. */
|
||||||
status = redisAsyncCommand(
|
status = redisAsyncCommand(
|
||||||
db->sub_context, object_table_redis_subscribe_to_notifications_callback,
|
db->sub_context, object_table_redis_subscribe_to_notifications_callback,
|
||||||
|
@ -684,7 +669,7 @@ void redis_object_table_request_notifications_callback(redisAsyncContext *c,
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
/* Do some minimal checking. */
|
/* Do some minimal checking. */
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECK(strcmp(reply->str, "OK") == 0);
|
CHECK(strcmp(reply->str, "OK") == 0);
|
||||||
CHECK(callback_data->done_callback == NULL);
|
CHECK(callback_data->done_callback == NULL);
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
|
@ -692,17 +677,18 @@ void redis_object_table_request_notifications_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_table_request_notifications(
|
void redis_object_table_request_notifications(
|
||||||
table_callback_data *callback_data) {
|
TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
|
|
||||||
object_table_request_notifications_data *request_data = callback_data->data;
|
ObjectTableRequestNotificationsData *request_data =
|
||||||
|
(ObjectTableRequestNotificationsData *) callback_data->data;
|
||||||
int num_object_ids = request_data->num_object_ids;
|
int num_object_ids = request_data->num_object_ids;
|
||||||
object_id *object_ids = request_data->object_ids;
|
ObjectID *object_ids = request_data->object_ids;
|
||||||
|
|
||||||
/* Create the arguments for the Redis command. */
|
/* Create the arguments for the Redis command. */
|
||||||
int num_args = 1 + 1 + num_object_ids;
|
int num_args = 1 + 1 + num_object_ids;
|
||||||
const char **argv = malloc(sizeof(char *) * num_args);
|
const char **argv = (const char **) malloc(sizeof(char *) * num_args);
|
||||||
size_t *argvlen = malloc(sizeof(size_t) * num_args);
|
size_t *argvlen = (size_t *) malloc(sizeof(size_t) * num_args);
|
||||||
/* Set the command name argument. */
|
/* Set the command name argument. */
|
||||||
argv[0] = "RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS";
|
argv[0] = "RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS";
|
||||||
argvlen[0] = strlen(argv[0]);
|
argvlen[0] = strlen(argv[0]);
|
||||||
|
@ -735,25 +721,26 @@ void redis_task_table_get_task_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
/* Parse the task from the reply. */
|
/* Parse the task from the reply. */
|
||||||
task *task = parse_and_construct_task_from_redis_reply(reply);
|
Task *task = parse_and_construct_task_from_redis_reply(reply);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
task_table_get_callback done_callback = callback_data->done_callback;
|
task_table_get_callback done_callback =
|
||||||
|
(task_table_get_callback) callback_data->done_callback;
|
||||||
if (done_callback != NULL) {
|
if (done_callback != NULL) {
|
||||||
done_callback(task, callback_data->user_context);
|
done_callback(task, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Free the task if it is not NULL. */
|
/* Free the task if it is not NULL. */
|
||||||
free_task(task);
|
Task_free(task);
|
||||||
|
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_task_table_get_task(table_callback_data *callback_data) {
|
void redis_task_table_get_task(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
CHECK(callback_data->data == NULL);
|
CHECK(callback_data->data == NULL);
|
||||||
task_id task_id = callback_data->id;
|
TaskID task_id = callback_data->id;
|
||||||
|
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, redis_task_table_get_task_callback,
|
db->context, redis_task_table_get_task_callback,
|
||||||
|
@ -770,24 +757,25 @@ void redis_task_table_add_task_callback(redisAsyncContext *c,
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
/* Do some minimal checking. */
|
/* Do some minimal checking. */
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECKM(strcmp(reply->str, "OK") == 0, "reply->str is %s", reply->str);
|
CHECKM(strcmp(reply->str, "OK") == 0, "reply->str is %s", reply->str);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
task_table_done_callback done_callback = callback_data->done_callback;
|
task_table_done_callback done_callback =
|
||||||
|
(task_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_task_table_add_task(table_callback_data *callback_data) {
|
void redis_task_table_add_task(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
task *task = callback_data->data;
|
Task *task = (Task *) callback_data->data;
|
||||||
task_id task_id = task_task_id(task);
|
TaskID task_id = Task_task_id(task);
|
||||||
db_client_id local_scheduler_id = task_local_scheduler(task);
|
DBClientID local_scheduler_id = Task_local_scheduler_id(task);
|
||||||
int state = task_state(task);
|
int state = Task_state(task);
|
||||||
task_spec *spec = task_task_spec(task);
|
task_spec *spec = Task_task_spec(task);
|
||||||
|
|
||||||
CHECKM(task != NULL, "NULL task passed to redis_task_table_add_task.");
|
CHECKM(task != NULL, "NULL task passed to redis_task_table_add_task.");
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
|
@ -806,23 +794,24 @@ void redis_task_table_update_callback(redisAsyncContext *c,
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
/* Do some minimal checking. */
|
/* Do some minimal checking. */
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECKM(strcmp(reply->str, "OK") == 0, "reply->str is %s", reply->str);
|
CHECKM(strcmp(reply->str, "OK") == 0, "reply->str is %s", reply->str);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
task_table_done_callback done_callback = callback_data->done_callback;
|
task_table_done_callback done_callback =
|
||||||
|
(task_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Clean up the timer and callback. */
|
/* Clean up the timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_task_table_update(table_callback_data *callback_data) {
|
void redis_task_table_update(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
task *task = callback_data->data;
|
Task *task = (Task *) callback_data->data;
|
||||||
task_id task_id = task_task_id(task);
|
TaskID task_id = Task_task_id(task);
|
||||||
db_client_id local_scheduler_id = task_local_scheduler(task);
|
DBClientID local_scheduler_id = Task_local_scheduler_id(task);
|
||||||
int state = task_state(task);
|
int state = Task_state(task);
|
||||||
|
|
||||||
CHECKM(task != NULL, "NULL task passed to redis_task_table_update.");
|
CHECKM(task != NULL, "NULL task passed to redis_task_table_update.");
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
|
@ -839,26 +828,28 @@ void redis_task_table_test_and_update_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
/* Parse the task from the reply. */
|
/* Parse the task from the reply. */
|
||||||
task *task = parse_and_construct_task_from_redis_reply(reply);
|
Task *task = parse_and_construct_task_from_redis_reply(reply);
|
||||||
/* Call the done callback if there is one. */
|
/* Call the done callback if there is one. */
|
||||||
task_table_get_callback done_callback = callback_data->done_callback;
|
task_table_get_callback done_callback =
|
||||||
|
(task_table_get_callback) callback_data->done_callback;
|
||||||
if (done_callback != NULL) {
|
if (done_callback != NULL) {
|
||||||
done_callback(task, callback_data->user_context);
|
done_callback(task, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Free the task if it is not NULL. */
|
/* Free the task if it is not NULL. */
|
||||||
if (task != NULL) {
|
if (task != NULL) {
|
||||||
free_task(task);
|
Task_free(task);
|
||||||
}
|
}
|
||||||
/* Clean up timer and callback. */
|
/* Clean up timer and callback. */
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_task_table_test_and_update(table_callback_data *callback_data) {
|
void redis_task_table_test_and_update(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
task_id task_id = callback_data->id;
|
TaskID task_id = callback_data->id;
|
||||||
task_table_test_and_update_data *update_data = callback_data->data;
|
TaskTableTestAndUpdateData *update_data =
|
||||||
|
(TaskTableTestAndUpdateData *) callback_data->data;
|
||||||
|
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, redis_task_table_test_and_update_callback,
|
db->context, redis_task_table_test_and_update_callback,
|
||||||
|
@ -877,9 +868,9 @@ void redis_task_table_test_and_update(table_callback_data *callback_data) {
|
||||||
* Make this code nicer. */
|
* Make this code nicer. */
|
||||||
void parse_task_table_subscribe_callback(char *payload,
|
void parse_task_table_subscribe_callback(char *payload,
|
||||||
int length,
|
int length,
|
||||||
task_id *task_id,
|
TaskID *task_id,
|
||||||
int *state,
|
int *state,
|
||||||
db_client_id *local_scheduler_id,
|
DBClientID *local_scheduler_id,
|
||||||
task_spec **spec) {
|
task_spec **spec) {
|
||||||
/* Note that the state is padded with spaces to consist of precisely two
|
/* Note that the state is padded with spaces to consist of precisely two
|
||||||
* characters. */
|
* characters. */
|
||||||
|
@ -890,7 +881,7 @@ void parse_task_table_subscribe_callback(char *payload,
|
||||||
memcpy(task_id, &payload[offset], sizeof(*task_id));
|
memcpy(task_id, &payload[offset], sizeof(*task_id));
|
||||||
offset += sizeof(*task_id);
|
offset += sizeof(*task_id);
|
||||||
/* Read in a space. */
|
/* Read in a space. */
|
||||||
char *space_str = " ";
|
const char *space_str = (const char *) " ";
|
||||||
CHECK(memcmp(space_str, &payload[offset], strlen(space_str)) == 0);
|
CHECK(memcmp(space_str, &payload[offset], strlen(space_str)) == 0);
|
||||||
offset += strlen(space_str);
|
offset += strlen(space_str);
|
||||||
/* Read in the state, which is an integer left-padded with spaces to two
|
/* Read in the state, which is an integer left-padded with spaces to two
|
||||||
|
@ -907,7 +898,7 @@ void parse_task_table_subscribe_callback(char *payload,
|
||||||
CHECK(memcmp(space_str, &payload[offset], strlen(space_str)) == 0);
|
CHECK(memcmp(space_str, &payload[offset], strlen(space_str)) == 0);
|
||||||
offset += strlen(space_str);
|
offset += strlen(space_str);
|
||||||
/* Read in the task spec. */
|
/* Read in the task spec. */
|
||||||
*spec = malloc(task_spec_payload_size);
|
*spec = (task_spec *) malloc(task_spec_payload_size);
|
||||||
memcpy(*spec, &payload[offset], task_spec_payload_size);
|
memcpy(*spec, &payload[offset], task_spec_payload_size);
|
||||||
CHECK(task_spec_size(*spec) == task_spec_payload_size);
|
CHECK(task_spec_size(*spec) == task_spec_payload_size);
|
||||||
}
|
}
|
||||||
|
@ -916,7 +907,7 @@ void redis_task_table_subscribe_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
|
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
/* The number of elements is 3 for a reply to SUBSCRIBE, and 4 for a reply to
|
/* The number of elements is 3 for a reply to SUBSCRIBE, and 4 for a reply to
|
||||||
|
@ -931,27 +922,29 @@ void redis_task_table_subscribe_callback(redisAsyncContext *c,
|
||||||
if (strcmp(message_type->str, "message") == 0 ||
|
if (strcmp(message_type->str, "message") == 0 ||
|
||||||
strcmp(message_type->str, "pmessage") == 0) {
|
strcmp(message_type->str, "pmessage") == 0) {
|
||||||
/* Handle a task table event. Parse the payload and call the callback. */
|
/* Handle a task table event. Parse the payload and call the callback. */
|
||||||
task_table_subscribe_data *data = callback_data->data;
|
TaskTableSubscribeData *data =
|
||||||
|
(TaskTableSubscribeData *) callback_data->data;
|
||||||
/* Read out the information from the payload. */
|
/* Read out the information from the payload. */
|
||||||
task_id task_id;
|
TaskID task_id;
|
||||||
int state;
|
int state;
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
task_spec *spec;
|
task_spec *spec;
|
||||||
parse_task_table_subscribe_callback(payload->str, payload->len, &task_id,
|
parse_task_table_subscribe_callback(payload->str, payload->len, &task_id,
|
||||||
&state, &local_scheduler_id, &spec);
|
&state, &local_scheduler_id, &spec);
|
||||||
task *task = alloc_task(spec, state, local_scheduler_id);
|
Task *task = Task_alloc(spec, state, local_scheduler_id);
|
||||||
free(spec);
|
free(spec);
|
||||||
/* Call the subscribe callback if there is one. */
|
/* Call the subscribe callback if there is one. */
|
||||||
if (data->subscribe_callback != NULL) {
|
if (data->subscribe_callback != NULL) {
|
||||||
data->subscribe_callback(task, data->subscribe_context);
|
data->subscribe_callback(task, data->subscribe_context);
|
||||||
}
|
}
|
||||||
free_task(task);
|
Task_free(task);
|
||||||
} else if (strcmp(message_type->str, "subscribe") == 0 ||
|
} else if (strcmp(message_type->str, "subscribe") == 0 ||
|
||||||
strcmp(message_type->str, "psubscribe") == 0) {
|
strcmp(message_type->str, "psubscribe") == 0) {
|
||||||
/* If this condition is true, we got the initial message that acknowledged
|
/* If this condition is true, we got the initial message that acknowledged
|
||||||
* the subscription. */
|
* the subscription. */
|
||||||
if (callback_data->done_callback != NULL) {
|
if (callback_data->done_callback != NULL) {
|
||||||
task_table_done_callback done_callback = callback_data->done_callback;
|
task_table_done_callback done_callback =
|
||||||
|
(task_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Note that we do not destroy the callback data yet because the
|
/* Note that we do not destroy the callback data yet because the
|
||||||
|
@ -964,9 +957,9 @@ void redis_task_table_subscribe_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_task_table_subscribe(table_callback_data *callback_data) {
|
void redis_task_table_subscribe(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
task_table_subscribe_data *data = callback_data->data;
|
TaskTableSubscribeData *data = (TaskTableSubscribeData *) callback_data->data;
|
||||||
/* TASK_CHANNEL_PREFIX is defined in ray_redis_module.c and must be kept in
|
/* TASK_CHANNEL_PREFIX is defined in ray_redis_module.c and must be kept in
|
||||||
* sync with that file. */
|
* sync with that file. */
|
||||||
const char *TASK_CHANNEL_PREFIX = "TT:";
|
const char *TASK_CHANNEL_PREFIX = "TT:";
|
||||||
|
@ -979,7 +972,7 @@ void redis_task_table_subscribe(table_callback_data *callback_data) {
|
||||||
(void *) callback_data->timer_id, "PSUBSCRIBE %s*:%2d",
|
(void *) callback_data->timer_id, "PSUBSCRIBE %s*:%2d",
|
||||||
TASK_CHANNEL_PREFIX, data->state_filter);
|
TASK_CHANNEL_PREFIX, data->state_filter);
|
||||||
} else {
|
} else {
|
||||||
db_client_id local_scheduler_id = data->local_scheduler_id;
|
DBClientID local_scheduler_id = data->local_scheduler_id;
|
||||||
status = redisAsyncCommand(
|
status = redisAsyncCommand(
|
||||||
db->sub_context, redis_task_table_subscribe_callback,
|
db->sub_context, redis_task_table_subscribe_callback,
|
||||||
(void *) callback_data->timer_id, "SUBSCRIBE %s%b:%2d",
|
(void *) callback_data->timer_id, "SUBSCRIBE %s%b:%2d",
|
||||||
|
@ -999,7 +992,7 @@ void redis_db_client_table_subscribe_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
|
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
CHECK(reply->elements > 2);
|
CHECK(reply->elements > 2);
|
||||||
|
@ -1011,7 +1004,7 @@ void redis_db_client_table_subscribe_callback(redisAsyncContext *c,
|
||||||
if (payload->str == NULL) {
|
if (payload->str == NULL) {
|
||||||
if (callback_data->done_callback) {
|
if (callback_data->done_callback) {
|
||||||
db_client_table_done_callback done_callback =
|
db_client_table_done_callback done_callback =
|
||||||
callback_data->done_callback;
|
(db_client_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Note that we do not destroy the callback data yet because the
|
/* Note that we do not destroy the callback data yet because the
|
||||||
|
@ -1020,14 +1013,15 @@ void redis_db_client_table_subscribe_callback(redisAsyncContext *c,
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
/* Otherwise, parse the payload and call the callback. */
|
/* Otherwise, parse the payload and call the callback. */
|
||||||
db_client_table_subscribe_data *data = callback_data->data;
|
DBClientTableSubscribeData *data =
|
||||||
db_client_id client;
|
(DBClientTableSubscribeData *) callback_data->data;
|
||||||
|
DBClientID client;
|
||||||
memcpy(client.id, payload->str, sizeof(client.id));
|
memcpy(client.id, payload->str, sizeof(client.id));
|
||||||
/* We subtract 1 + sizeof(client.id) to compute the length of the
|
/* We subtract 1 + sizeof(client.id) to compute the length of the
|
||||||
* client_type string, and we add 1 to null-terminate the string. */
|
* client_type string, and we add 1 to null-terminate the string. */
|
||||||
int client_type_length = payload->len - 1 - sizeof(client.id) + 1;
|
int client_type_length = payload->len - 1 - sizeof(client.id) + 1;
|
||||||
char *client_type = malloc(client_type_length);
|
char *client_type = (char *) malloc(client_type_length);
|
||||||
char *aux_address = malloc(client_type_length);
|
char *aux_address = (char *) malloc(client_type_length);
|
||||||
memset(aux_address, 0, client_type_length);
|
memset(aux_address, 0, client_type_length);
|
||||||
/* Published message format: <client_id:client_type aux_addr> */
|
/* Published message format: <client_id:client_type aux_addr> */
|
||||||
int rv = sscanf(&payload->str[1 + sizeof(client.id)], "%s %s", client_type,
|
int rv = sscanf(&payload->str[1 + sizeof(client.id)], "%s %s", client_type,
|
||||||
|
@ -1044,8 +1038,8 @@ void redis_db_client_table_subscribe_callback(redisAsyncContext *c,
|
||||||
free(aux_address);
|
free(aux_address);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_db_client_table_subscribe(table_callback_data *callback_data) {
|
void redis_db_client_table_subscribe(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->sub_context, redis_db_client_table_subscribe_callback,
|
db->sub_context, redis_db_client_table_subscribe_callback,
|
||||||
(void *) callback_data->timer_id, "SUBSCRIBE db_clients");
|
(void *) callback_data->timer_id, "SUBSCRIBE db_clients");
|
||||||
|
@ -1060,7 +1054,7 @@ void redis_local_scheduler_table_subscribe_callback(redisAsyncContext *c,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
CHECK(reply->elements == 3);
|
CHECK(reply->elements == 3);
|
||||||
redisReply *message_type = reply->element[0];
|
redisReply *message_type = reply->element[0];
|
||||||
|
@ -1071,9 +1065,10 @@ void redis_local_scheduler_table_subscribe_callback(redisAsyncContext *c,
|
||||||
/* Handle a local scheduler heartbeat. Parse the payload and call the
|
/* Handle a local scheduler heartbeat. Parse the payload and call the
|
||||||
* subscribe callback. */
|
* subscribe callback. */
|
||||||
redisReply *payload = reply->element[2];
|
redisReply *payload = reply->element[2];
|
||||||
local_scheduler_table_subscribe_data *data = callback_data->data;
|
LocalSchedulerTableSubscribeData *data =
|
||||||
db_client_id client_id;
|
(LocalSchedulerTableSubscribeData *) callback_data->data;
|
||||||
local_scheduler_info info;
|
DBClientID client_id;
|
||||||
|
LocalSchedulerInfo info;
|
||||||
/* The payload should be the concatenation of these two structs. */
|
/* The payload should be the concatenation of these two structs. */
|
||||||
CHECK(sizeof(client_id) + sizeof(info) == payload->len);
|
CHECK(sizeof(client_id) + sizeof(info) == payload->len);
|
||||||
memcpy(&client_id, payload->str, sizeof(client_id));
|
memcpy(&client_id, payload->str, sizeof(client_id));
|
||||||
|
@ -1093,8 +1088,8 @@ void redis_local_scheduler_table_subscribe_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_local_scheduler_table_subscribe(table_callback_data *callback_data) {
|
void redis_local_scheduler_table_subscribe(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->sub_context, redis_local_scheduler_table_subscribe_callback,
|
db->sub_context, redis_local_scheduler_table_subscribe_callback,
|
||||||
(void *) callback_data->timer_id, "SUBSCRIBE local_schedulers");
|
(void *) callback_data->timer_id, "SUBSCRIBE local_schedulers");
|
||||||
|
@ -1109,7 +1104,7 @@ void redis_local_scheduler_table_send_info_callback(redisAsyncContext *c,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECK(reply->type == REDIS_REPLY_INTEGER);
|
CHECK(reply->type == REDIS_REPLY_INTEGER);
|
||||||
LOG_DEBUG("%" PRId64 " subscribers received this publish.\n", reply->integer);
|
LOG_DEBUG("%" PRId64 " subscribers received this publish.\n", reply->integer);
|
||||||
|
|
||||||
|
@ -1118,9 +1113,10 @@ void redis_local_scheduler_table_send_info_callback(redisAsyncContext *c,
|
||||||
destroy_timer_callback(db->loop, callback_data);
|
destroy_timer_callback(db->loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_local_scheduler_table_send_info(table_callback_data *callback_data) {
|
void redis_local_scheduler_table_send_info(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
local_scheduler_table_send_info_data *data = callback_data->data;
|
LocalSchedulerTableSendInfoData *data =
|
||||||
|
(LocalSchedulerTableSendInfoData *) callback_data->data;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->context, redis_local_scheduler_table_send_info_callback,
|
db->context, redis_local_scheduler_table_send_info_callback,
|
||||||
(void *) callback_data->timer_id, "PUBLISH local_schedulers %b%b",
|
(void *) callback_data->timer_id, "PUBLISH local_schedulers %b%b",
|
||||||
|
@ -1136,7 +1132,7 @@ void redis_actor_notification_table_subscribe_callback(redisAsyncContext *c,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
|
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
CHECK(reply->elements == 3);
|
CHECK(reply->elements == 3);
|
||||||
redisReply *message_type = reply->element[0];
|
redisReply *message_type = reply->element[0];
|
||||||
|
@ -1147,8 +1143,9 @@ void redis_actor_notification_table_subscribe_callback(redisAsyncContext *c,
|
||||||
/* Handle an actor notification message. Parse the payload and call the
|
/* Handle an actor notification message. Parse the payload and call the
|
||||||
* subscribe callback. */
|
* subscribe callback. */
|
||||||
redisReply *payload = reply->element[2];
|
redisReply *payload = reply->element[2];
|
||||||
actor_notification_table_subscribe_data *data = callback_data->data;
|
ActorNotificationTableSubscribeData *data =
|
||||||
actor_info info;
|
(ActorNotificationTableSubscribeData *) callback_data->data;
|
||||||
|
ActorInfo info;
|
||||||
/* The payload should be the concatenation of these two structs. */
|
/* The payload should be the concatenation of these two structs. */
|
||||||
CHECK(sizeof(info.actor_id) + sizeof(info.local_scheduler_id) ==
|
CHECK(sizeof(info.actor_id) + sizeof(info.local_scheduler_id) ==
|
||||||
payload->len);
|
payload->len);
|
||||||
|
@ -1171,8 +1168,8 @@ void redis_actor_notification_table_subscribe_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_actor_notification_table_subscribe(
|
void redis_actor_notification_table_subscribe(
|
||||||
table_callback_data *callback_data) {
|
TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->sub_context, redis_actor_notification_table_subscribe_callback,
|
db->sub_context, redis_actor_notification_table_subscribe_callback,
|
||||||
(void *) callback_data->timer_id, "SUBSCRIBE actor_notifications");
|
(void *) callback_data->timer_id, "SUBSCRIBE actor_notifications");
|
||||||
|
@ -1186,7 +1183,7 @@ void redis_object_info_subscribe_callback(redisAsyncContext *c,
|
||||||
void *r,
|
void *r,
|
||||||
void *privdata) {
|
void *privdata) {
|
||||||
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
REDIS_CALLBACK_HEADER(db, callback_data, r);
|
||||||
redisReply *reply = r;
|
redisReply *reply = (redisReply *) r;
|
||||||
|
|
||||||
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
CHECK(reply->type == REDIS_REPLY_ARRAY);
|
||||||
|
|
||||||
|
@ -1199,7 +1196,7 @@ void redis_object_info_subscribe_callback(redisAsyncContext *c,
|
||||||
if (payload->str == NULL) {
|
if (payload->str == NULL) {
|
||||||
if (callback_data->done_callback) {
|
if (callback_data->done_callback) {
|
||||||
db_client_table_done_callback done_callback =
|
db_client_table_done_callback done_callback =
|
||||||
callback_data->done_callback;
|
(db_client_table_done_callback) callback_data->done_callback;
|
||||||
done_callback(callback_data->id, callback_data->user_context);
|
done_callback(callback_data->id, callback_data->user_context);
|
||||||
}
|
}
|
||||||
/* Note that we do not destroy the callback data yet because the
|
/* Note that we do not destroy the callback data yet because the
|
||||||
|
@ -1208,10 +1205,11 @@ void redis_object_info_subscribe_callback(redisAsyncContext *c,
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
/* Otherwise, parse the payload and call the callback. */
|
/* Otherwise, parse the payload and call the callback. */
|
||||||
object_info_subscribe_data *data = callback_data->data;
|
ObjectInfoSubscribeData *data =
|
||||||
object_id object_id;
|
(ObjectInfoSubscribeData *) callback_data->data;
|
||||||
|
ObjectID object_id;
|
||||||
memcpy(object_id.id, payload->str, sizeof(object_id.id));
|
memcpy(object_id.id, payload->str, sizeof(object_id.id));
|
||||||
/* payload->str should have the format: "object_id:object_size_int" */
|
/* payload->str should have the format: "ObjectID:object_size_int" */
|
||||||
LOG_DEBUG("obj:info channel received message <%s>", payload->str);
|
LOG_DEBUG("obj:info channel received message <%s>", payload->str);
|
||||||
if (data->subscribe_callback) {
|
if (data->subscribe_callback) {
|
||||||
data->subscribe_callback(
|
data->subscribe_callback(
|
||||||
|
@ -1220,8 +1218,8 @@ void redis_object_info_subscribe_callback(redisAsyncContext *c,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_object_info_subscribe(table_callback_data *callback_data) {
|
void redis_object_info_subscribe(TableCallbackData *callback_data) {
|
||||||
db_handle *db = callback_data->db_handle;
|
DBHandle *db = callback_data->db_handle;
|
||||||
int status = redisAsyncCommand(
|
int status = redisAsyncCommand(
|
||||||
db->sub_context, redis_object_info_subscribe_callback,
|
db->sub_context, redis_object_info_subscribe_callback,
|
||||||
(void *) callback_data->timer_id, "PSUBSCRIBE obj:info");
|
(void *) callback_data->timer_id, "PSUBSCRIBE obj:info");
|
||||||
|
@ -1230,7 +1228,7 @@ void redis_object_info_subscribe(table_callback_data *callback_data) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
db_client_id get_db_client_id(db_handle *db) {
|
DBClientID get_db_client_id(DBHandle *db) {
|
||||||
CHECK(db != NULL);
|
CHECK(db != NULL);
|
||||||
return db->client;
|
return db->client;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,18 +22,18 @@
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Unique ID for this db client. */
|
/** Unique ID for this db client. */
|
||||||
db_client_id db_client_id;
|
DBClientID db_client_id;
|
||||||
/** IP address and port of this db client. */
|
/** IP address and port of this db client. */
|
||||||
char *addr;
|
char *addr;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} db_client_cache_entry;
|
} DBClientCacheEntry;
|
||||||
|
|
||||||
struct db_handle {
|
struct DBHandle {
|
||||||
/** String that identifies this client type. */
|
/** String that identifies this client type. */
|
||||||
char *client_type;
|
char *client_type;
|
||||||
/** Unique ID for this client. */
|
/** Unique ID for this client. */
|
||||||
db_client_id client;
|
DBClientID client;
|
||||||
/** Redis context for all non-subscribe connections. */
|
/** Redis context for all non-subscribe connections. */
|
||||||
redisAsyncContext *context;
|
redisAsyncContext *context;
|
||||||
/** Redis context for "subscribe" communication. Yes, we need a separate one
|
/** Redis context for "subscribe" communication. Yes, we need a separate one
|
||||||
|
@ -45,7 +45,7 @@ struct db_handle {
|
||||||
int64_t db_index;
|
int64_t db_index;
|
||||||
/** Cache for the IP addresses of db clients. This is a hash table mapping
|
/** Cache for the IP addresses of db clients. This is a hash table mapping
|
||||||
* client IDs to addresses. */
|
* client IDs to addresses. */
|
||||||
db_client_cache_entry *db_client_cache;
|
DBClientCacheEntry *db_client_cache;
|
||||||
/** Redis context for synchronous connections. This should only be used very
|
/** Redis context for synchronous connections. This should only be used very
|
||||||
* rarely, it is not asynchronous. */
|
* rarely, it is not asynchronous. */
|
||||||
redisContext *sync_context;
|
redisContext *sync_context;
|
||||||
|
@ -70,7 +70,7 @@ void object_table_lookup_callback(redisAsyncContext *c,
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_object_table_lookup(table_callback_data *callback_data);
|
void redis_object_table_lookup(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a location entry to the object table in redis.
|
* Add a location entry to the object table in redis.
|
||||||
|
@ -79,7 +79,7 @@ void redis_object_table_lookup(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_object_table_add(table_callback_data *callback_data);
|
void redis_object_table_add(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove a location entry from the object table in redis.
|
* Remove a location entry from the object table in redis.
|
||||||
|
@ -88,7 +88,7 @@ void redis_object_table_add(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_object_table_remove(table_callback_data *callback_data);
|
void redis_object_table_remove(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a client-specific channel for receiving notifications from the object
|
* Create a client-specific channel for receiving notifications from the object
|
||||||
|
@ -99,7 +99,7 @@ void redis_object_table_remove(table_callback_data *callback_data);
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_object_table_subscribe_to_notifications(
|
void redis_object_table_subscribe_to_notifications(
|
||||||
table_callback_data *callback_data);
|
TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Request notifications about when certain objects become available.
|
* Request notifications about when certain objects become available.
|
||||||
|
@ -108,8 +108,7 @@ void redis_object_table_subscribe_to_notifications(
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_object_table_request_notifications(
|
void redis_object_table_request_notifications(TableCallbackData *callback_data);
|
||||||
table_callback_data *callback_data);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a new object to the object table in redis.
|
* Add a new object to the object table in redis.
|
||||||
|
@ -118,7 +117,7 @@ void redis_object_table_request_notifications(
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_result_table_add(table_callback_data *callback_data);
|
void redis_result_table_add(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Lookup the task that created the object in redis. The result is the task ID.
|
* Lookup the task that created the object in redis. The result is the task ID.
|
||||||
|
@ -127,7 +126,7 @@ void redis_result_table_add(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_result_table_lookup(table_callback_data *callback_data);
|
void redis_result_table_lookup(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Callback invoked when the reply from the object table lookup command is
|
* Callback invoked when the reply from the object table lookup command is
|
||||||
|
@ -154,7 +153,7 @@ void redis_object_table_lookup_callback(redisAsyncContext *c,
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_task_table_get_task(table_callback_data *callback_data);
|
void redis_task_table_get_task(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a task table entry with a new task spec and the task's scheduling
|
* Add a task table entry with a new task spec and the task's scheduling
|
||||||
|
@ -164,7 +163,7 @@ void redis_task_table_get_task(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_task_table_add_task(table_callback_data *callback_data);
|
void redis_task_table_add_task(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update a task table entry with the task's scheduling information.
|
* Update a task table entry with the task's scheduling information.
|
||||||
|
@ -173,7 +172,7 @@ void redis_task_table_add_task(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_task_table_update(table_callback_data *callback_data);
|
void redis_task_table_update(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update a task table entry with the task's scheduling information, if the
|
* Update a task table entry with the task's scheduling information, if the
|
||||||
|
@ -183,7 +182,7 @@ void redis_task_table_update(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_task_table_test_and_update(table_callback_data *callback_data);
|
void redis_task_table_test_and_update(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Callback invoked when the reply from the task push command is received.
|
* Callback invoked when the reply from the task push command is received.
|
||||||
|
@ -216,7 +215,7 @@ void redis_task_table_publish_publish_callback(redisAsyncContext *c,
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_task_table_subscribe(table_callback_data *callback_data);
|
void redis_task_table_subscribe(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subscribe to updates from the db client table.
|
* Subscribe to updates from the db client table.
|
||||||
|
@ -225,7 +224,7 @@ void redis_task_table_subscribe(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_db_client_table_subscribe(table_callback_data *callback_data);
|
void redis_db_client_table_subscribe(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subscribe to updates from the local scheduler table.
|
* Subscribe to updates from the local scheduler table.
|
||||||
|
@ -234,7 +233,7 @@ void redis_db_client_table_subscribe(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_local_scheduler_table_subscribe(table_callback_data *callback_data);
|
void redis_local_scheduler_table_subscribe(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Publish an update to the local scheduler table.
|
* Publish an update to the local scheduler table.
|
||||||
|
@ -243,7 +242,7 @@ void redis_local_scheduler_table_subscribe(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_local_scheduler_table_send_info(table_callback_data *callback_data);
|
void redis_local_scheduler_table_send_info(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subscribe to updates about newly created actors.
|
* Subscribe to updates about newly created actors.
|
||||||
|
@ -252,9 +251,8 @@ void redis_local_scheduler_table_send_info(table_callback_data *callback_data);
|
||||||
* information.
|
* information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void redis_actor_notification_table_subscribe(
|
void redis_actor_notification_table_subscribe(TableCallbackData *callback_data);
|
||||||
table_callback_data *callback_data);
|
|
||||||
|
|
||||||
void redis_object_info_subscribe(table_callback_data *callback_data);
|
void redis_object_info_subscribe(TableCallbackData *callback_data);
|
||||||
|
|
||||||
#endif /* REDIS_H */
|
#endif /* REDIS_H */
|
||||||
|
|
|
@ -4,27 +4,27 @@
|
||||||
#include "redis.h"
|
#include "redis.h"
|
||||||
|
|
||||||
/* The default behavior is to retry every ten seconds forever. */
|
/* The default behavior is to retry every ten seconds forever. */
|
||||||
static const retry_info default_retry = {.num_retries = -1,
|
static const RetryInfo default_retry = {.num_retries = -1,
|
||||||
.timeout = 10000,
|
.timeout = 10000,
|
||||||
.fail_callback = NULL};
|
.fail_callback = NULL};
|
||||||
|
|
||||||
table_callback_data *init_table_callback(db_handle *db_handle,
|
TableCallbackData *init_table_callback(DBHandle *db_handle,
|
||||||
unique_id id,
|
UniqueID id,
|
||||||
const char *label,
|
const char *label,
|
||||||
OWNER void *data,
|
OWNER void *data,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
table_done_callback done_callback,
|
table_done_callback done_callback,
|
||||||
table_retry_callback retry_callback,
|
table_retry_callback retry_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(db_handle);
|
CHECK(db_handle);
|
||||||
CHECK(db_handle->loop);
|
CHECK(db_handle->loop);
|
||||||
/* If no retry info is provided, use the default retry info. */
|
/* If no retry info is provided, use the default retry info. */
|
||||||
if (retry == NULL) {
|
if (retry == NULL) {
|
||||||
retry = (retry_info *) &default_retry;
|
retry = (RetryInfo *) &default_retry;
|
||||||
}
|
}
|
||||||
CHECK(retry);
|
CHECK(retry);
|
||||||
/* Allocate and initialize callback data structure for object table */
|
/* Allocate and initialize callback data structure for object table */
|
||||||
table_callback_data *callback_data = malloc(sizeof(table_callback_data));
|
TableCallbackData *callback_data = malloc(sizeof(TableCallbackData));
|
||||||
CHECKM(callback_data != NULL, "Memory allocation error!")
|
CHECKM(callback_data != NULL, "Memory allocation error!")
|
||||||
callback_data->id = id;
|
callback_data->id = id;
|
||||||
callback_data->label = label;
|
callback_data->label = label;
|
||||||
|
@ -49,12 +49,12 @@ table_callback_data *init_table_callback(db_handle *db_handle,
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_timer_callback(event_loop *loop,
|
void destroy_timer_callback(event_loop *loop,
|
||||||
table_callback_data *callback_data) {
|
TableCallbackData *callback_data) {
|
||||||
event_loop_remove_timer(loop, callback_data->timer_id);
|
event_loop_remove_timer(loop, callback_data->timer_id);
|
||||||
destroy_table_callback(callback_data);
|
destroy_table_callback(callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_table_callback(table_callback_data *callback_data) {
|
void destroy_table_callback(TableCallbackData *callback_data) {
|
||||||
CHECK(callback_data != NULL);
|
CHECK(callback_data != NULL);
|
||||||
|
|
||||||
if (callback_data->requests_info)
|
if (callback_data->requests_info)
|
||||||
|
@ -76,7 +76,7 @@ int64_t table_timeout_handler(event_loop *loop,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(loop != NULL);
|
CHECK(loop != NULL);
|
||||||
CHECK(user_context != NULL);
|
CHECK(user_context != NULL);
|
||||||
table_callback_data *callback_data = (table_callback_data *) user_context;
|
TableCallbackData *callback_data = (TableCallbackData *) user_context;
|
||||||
|
|
||||||
CHECK(callback_data->retry.num_retries >= 0 ||
|
CHECK(callback_data->retry.num_retries >= 0 ||
|
||||||
callback_data->retry.num_retries == -1);
|
callback_data->retry.num_retries == -1);
|
||||||
|
@ -133,24 +133,24 @@ int64_t table_timeout_handler(event_loop *loop,
|
||||||
* When the last timeout associated to the command expires we remove the entry
|
* When the last timeout associated to the command expires we remove the entry
|
||||||
* associated to the callback.
|
* associated to the callback.
|
||||||
*/
|
*/
|
||||||
static table_callback_data *outstanding_callbacks = NULL;
|
static TableCallbackData *outstanding_callbacks = NULL;
|
||||||
|
|
||||||
void outstanding_callbacks_add(table_callback_data *callback_data) {
|
void outstanding_callbacks_add(TableCallbackData *callback_data) {
|
||||||
HASH_ADD_INT(outstanding_callbacks, timer_id, callback_data);
|
HASH_ADD_INT(outstanding_callbacks, timer_id, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
table_callback_data *outstanding_callbacks_find(int64_t key) {
|
TableCallbackData *outstanding_callbacks_find(int64_t key) {
|
||||||
table_callback_data *callback_data = NULL;
|
TableCallbackData *callback_data = NULL;
|
||||||
HASH_FIND_INT(outstanding_callbacks, &key, callback_data);
|
HASH_FIND_INT(outstanding_callbacks, &key, callback_data);
|
||||||
return callback_data;
|
return callback_data;
|
||||||
}
|
}
|
||||||
|
|
||||||
void outstanding_callbacks_remove(table_callback_data *callback_data) {
|
void outstanding_callbacks_remove(TableCallbackData *callback_data) {
|
||||||
HASH_DEL(outstanding_callbacks, callback_data);
|
HASH_DEL(outstanding_callbacks, callback_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_outstanding_callbacks(event_loop *loop) {
|
void destroy_outstanding_callbacks(event_loop *loop) {
|
||||||
table_callback_data *callback_data, *tmp;
|
TableCallbackData *callback_data, *tmp;
|
||||||
HASH_ITER(hh, outstanding_callbacks, callback_data, tmp) {
|
HASH_ITER(hh, outstanding_callbacks, callback_data, tmp) {
|
||||||
destroy_timer_callback(loop, callback_data);
|
destroy_timer_callback(loop, callback_data);
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,7 +7,7 @@
|
||||||
#include "common.h"
|
#include "common.h"
|
||||||
#include "db.h"
|
#include "db.h"
|
||||||
|
|
||||||
typedef struct table_callback_data table_callback_data;
|
typedef struct TableCallbackData TableCallbackData;
|
||||||
|
|
||||||
typedef void *table_done_callback;
|
typedef void *table_done_callback;
|
||||||
|
|
||||||
|
@ -17,20 +17,20 @@ typedef void *table_done_callback;
|
||||||
* @param id The unique ID that identifies this callback. Examples include an
|
* @param id The unique ID that identifies this callback. Examples include an
|
||||||
* object ID or task ID.
|
* object ID or task ID.
|
||||||
* @param user_context The state context for the callback. This is equivalent
|
* @param user_context The state context for the callback. This is equivalent
|
||||||
* to the user_context field in table_callback_data.
|
* to the user_context field in TableCallbackData.
|
||||||
* @param user_data A data argument for the callback. This is equivalent to the
|
* @param user_data A data argument for the callback. This is equivalent to the
|
||||||
* data field in table_callback_data. The user is responsible for
|
* data field in TableCallbackData. The user is responsible for
|
||||||
* freeing user_data.
|
* freeing user_data.
|
||||||
*/
|
*/
|
||||||
typedef void (*table_fail_callback)(unique_id id,
|
typedef void (*table_fail_callback)(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data);
|
void *user_data);
|
||||||
|
|
||||||
typedef void (*table_retry_callback)(table_callback_data *callback_data);
|
typedef void (*table_retry_callback)(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Data structure consolidating the retry related variables. If a NULL
|
* Data structure consolidating the retry related variables. If a NULL
|
||||||
* retry_info struct is used, the default behavior will be to retry infinitely
|
* RetryInfo struct is used, the default behavior will be to retry infinitely
|
||||||
* many times.
|
* many times.
|
||||||
*/
|
*/
|
||||||
typedef struct {
|
typedef struct {
|
||||||
|
@ -42,12 +42,12 @@ typedef struct {
|
||||||
uint64_t timeout;
|
uint64_t timeout;
|
||||||
/** The callback that will be called if there are no more retries left. */
|
/** The callback that will be called if there are no more retries left. */
|
||||||
table_fail_callback fail_callback;
|
table_fail_callback fail_callback;
|
||||||
} retry_info;
|
} RetryInfo;
|
||||||
|
|
||||||
struct table_callback_data {
|
struct TableCallbackData {
|
||||||
/** ID of the entry in the table that we are going to look up, remove or add.
|
/** ID of the entry in the table that we are going to look up, remove or add.
|
||||||
*/
|
*/
|
||||||
unique_id id;
|
UniqueID id;
|
||||||
/** A label to identify the original request for logging purposes. */
|
/** A label to identify the original request for logging purposes. */
|
||||||
const char *label;
|
const char *label;
|
||||||
/** The callback that will be called when results is returned. */
|
/** The callback that will be called when results is returned. */
|
||||||
|
@ -57,7 +57,7 @@ struct table_callback_data {
|
||||||
/** Retry information containing the remaining number of retries, the timeout
|
/** Retry information containing the remaining number of retries, the timeout
|
||||||
* before the next retry, and a pointer to the failure callback.
|
* before the next retry, and a pointer to the failure callback.
|
||||||
*/
|
*/
|
||||||
retry_info retry;
|
RetryInfo retry;
|
||||||
/** Pointer to the data that is entered into the table. This can be used to
|
/** Pointer to the data that is entered into the table. This can be used to
|
||||||
* pass the result of the call to the callback. The callback takes ownership
|
* pass the result of the call to the callback. The callback takes ownership
|
||||||
* over this data and will free it. */
|
* over this data and will free it. */
|
||||||
|
@ -68,7 +68,7 @@ struct table_callback_data {
|
||||||
/** User context. */
|
/** User context. */
|
||||||
void *user_context;
|
void *user_context;
|
||||||
/** Handle to db. */
|
/** Handle to db. */
|
||||||
db_handle *db_handle;
|
DBHandle *db_handle;
|
||||||
/** Handle to timer. */
|
/** Handle to timer. */
|
||||||
int64_t timer_id;
|
int64_t timer_id;
|
||||||
UT_hash_handle hh; /* makes this structure hashable */
|
UT_hash_handle hh; /* makes this structure hashable */
|
||||||
|
@ -104,14 +104,14 @@ int64_t table_timeout_handler(event_loop *loop,
|
||||||
* passed on to the various callbacks.
|
* passed on to the various callbacks.
|
||||||
* @return New table callback data struct.
|
* @return New table callback data struct.
|
||||||
*/
|
*/
|
||||||
table_callback_data *init_table_callback(db_handle *db_handle,
|
TableCallbackData *init_table_callback(DBHandle *db_handle,
|
||||||
unique_id id,
|
UniqueID id,
|
||||||
const char *label,
|
const char *label,
|
||||||
OWNER void *data,
|
OWNER void *data,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
table_done_callback done_callback,
|
table_done_callback done_callback,
|
||||||
table_retry_callback retry_callback,
|
table_retry_callback retry_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Destroy any state associated with the callback data. This removes all
|
* Destroy any state associated with the callback data. This removes all
|
||||||
|
@ -122,7 +122,7 @@ table_callback_data *init_table_callback(db_handle *db_handle,
|
||||||
* want to remove.
|
* want to remove.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void destroy_table_callback(table_callback_data *callback_data);
|
void destroy_table_callback(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Destroy all state events associated with the callback data, including memory
|
* Destroy all state events associated with the callback data, including memory
|
||||||
|
@ -132,8 +132,7 @@ void destroy_table_callback(table_callback_data *callback_data);
|
||||||
* want to remove.
|
* want to remove.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void destroy_timer_callback(event_loop *loop,
|
void destroy_timer_callback(event_loop *loop, TableCallbackData *callback_data);
|
||||||
table_callback_data *callback_data);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add an outstanding callback entry.
|
* Add an outstanding callback entry.
|
||||||
|
@ -142,7 +141,7 @@ void destroy_timer_callback(event_loop *loop,
|
||||||
* want to insert.
|
* want to insert.
|
||||||
* @return None.
|
* @return None.
|
||||||
*/
|
*/
|
||||||
void outstanding_callbacks_add(table_callback_data *callback_data);
|
void outstanding_callbacks_add(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Find an outstanding callback entry.
|
* Find an outstanding callback entry.
|
||||||
|
@ -151,7 +150,7 @@ void outstanding_callbacks_add(table_callback_data *callback_data);
|
||||||
* timer ID assigned by the Redis ae event loop.
|
* timer ID assigned by the Redis ae event loop.
|
||||||
* @return Returns the callback data if found, NULL otherwise.
|
* @return Returns the callback data if found, NULL otherwise.
|
||||||
*/
|
*/
|
||||||
table_callback_data *outstanding_callbacks_find(int64_t key);
|
TableCallbackData *outstanding_callbacks_find(int64_t key);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove an outstanding callback entry. This only removes the callback entry
|
* Remove an outstanding callback entry. This only removes the callback entry
|
||||||
|
@ -162,7 +161,7 @@ table_callback_data *outstanding_callbacks_find(int64_t key);
|
||||||
* want to remove.
|
* want to remove.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void outstanding_callbacks_remove(table_callback_data *callback_data);
|
void outstanding_callbacks_remove(TableCallbackData *callback_data);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Destroy all outstanding callbacks and remove their associated timer events
|
* Destroy all outstanding callbacks and remove their associated timer events
|
||||||
|
|
|
@ -3,42 +3,42 @@
|
||||||
|
|
||||||
#define NUM_DB_REQUESTS 2
|
#define NUM_DB_REQUESTS 2
|
||||||
|
|
||||||
void task_table_get_task(db_handle *db_handle,
|
void task_table_get_task(DBHandle *db_handle,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_get_callback done_callback,
|
task_table_get_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
init_table_callback(db_handle, task_id, __func__, NULL, retry, done_callback,
|
init_table_callback(db_handle, task_id, __func__, NULL, retry, done_callback,
|
||||||
redis_task_table_get_task, user_context);
|
redis_task_table_get_task, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_table_add_task(db_handle *db_handle,
|
void task_table_add_task(DBHandle *db_handle,
|
||||||
OWNER task *task,
|
OWNER Task *task,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
init_table_callback(db_handle, task_task_id(task), __func__, task, retry,
|
init_table_callback(db_handle, Task_task_id(task), __func__, task, retry,
|
||||||
done_callback, redis_task_table_add_task, user_context);
|
done_callback, redis_task_table_add_task, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_table_update(db_handle *db_handle,
|
void task_table_update(DBHandle *db_handle,
|
||||||
OWNER task *task,
|
OWNER Task *task,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
init_table_callback(db_handle, task_task_id(task), __func__, task, retry,
|
init_table_callback(db_handle, Task_task_id(task), __func__, task, retry,
|
||||||
done_callback, redis_task_table_update, user_context);
|
done_callback, redis_task_table_update, user_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_table_test_and_update(db_handle *db_handle,
|
void task_table_test_and_update(DBHandle *db_handle,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
int test_state_bitmask,
|
int test_state_bitmask,
|
||||||
int update_state,
|
int update_state,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_get_callback done_callback,
|
task_table_get_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
task_table_test_and_update_data *update_data =
|
TaskTableTestAndUpdateData *update_data =
|
||||||
malloc(sizeof(task_table_test_and_update_data));
|
malloc(sizeof(TaskTableTestAndUpdateData));
|
||||||
update_data->test_state_bitmask = test_state_bitmask;
|
update_data->test_state_bitmask = test_state_bitmask;
|
||||||
update_data->update_state = update_state;
|
update_data->update_state = update_state;
|
||||||
/* Update the task entry's local scheduler with this client's ID. */
|
/* Update the task entry's local scheduler with this client's ID. */
|
||||||
|
@ -49,16 +49,15 @@ void task_table_test_and_update(db_handle *db_handle,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* TODO(swang): A corresponding task_table_unsubscribe. */
|
/* TODO(swang): A corresponding task_table_unsubscribe. */
|
||||||
void task_table_subscribe(db_handle *db_handle,
|
void task_table_subscribe(DBHandle *db_handle,
|
||||||
db_client_id local_scheduler_id,
|
DBClientID local_scheduler_id,
|
||||||
int state_filter,
|
int state_filter,
|
||||||
task_table_subscribe_callback subscribe_callback,
|
task_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
task_table_subscribe_data *sub_data =
|
TaskTableSubscribeData *sub_data = malloc(sizeof(TaskTableSubscribeData));
|
||||||
malloc(sizeof(task_table_subscribe_data));
|
|
||||||
sub_data->local_scheduler_id = local_scheduler_id;
|
sub_data->local_scheduler_id = local_scheduler_id;
|
||||||
sub_data->state_filter = state_filter;
|
sub_data->state_filter = state_filter;
|
||||||
sub_data->subscribe_callback = subscribe_callback;
|
sub_data->subscribe_callback = subscribe_callback;
|
||||||
|
|
|
@ -22,11 +22,11 @@
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* Callback called when a task table write operation completes. */
|
/* Callback called when a task table write operation completes. */
|
||||||
typedef void (*task_table_done_callback)(task_id task_id, void *user_context);
|
typedef void (*task_table_done_callback)(TaskID task_id, void *user_context);
|
||||||
|
|
||||||
/* Callback called when a task table read operation completes. If the task ID
|
/* Callback called when a task table read operation completes. If the task ID
|
||||||
* was not in the task table, then the task pointer will be NULL. */
|
* was not in the task table, then the task pointer will be NULL. */
|
||||||
typedef void (*task_table_get_callback)(task *task, void *user_context);
|
typedef void (*task_table_get_callback)(Task *task, void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a task's entry from the task table.
|
* Get a task's entry from the task table.
|
||||||
|
@ -39,9 +39,9 @@ typedef void (*task_table_get_callback)(task *task, void *user_context);
|
||||||
* fail_callback.
|
* fail_callback.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void task_table_get_task(db_handle *db,
|
void task_table_get_task(DBHandle *db,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_get_callback done_callback,
|
task_table_get_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -58,9 +58,9 @@ void task_table_get_task(db_handle *db,
|
||||||
* fail_callback.
|
* fail_callback.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void task_table_add_task(db_handle *db_handle,
|
void task_table_add_task(DBHandle *db_handle,
|
||||||
OWNER task *task,
|
OWNER Task *task,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -81,9 +81,9 @@ void task_table_add_task(db_handle *db_handle,
|
||||||
* fail_callback.
|
* fail_callback.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void task_table_update(db_handle *db_handle,
|
void task_table_update(DBHandle *db_handle,
|
||||||
OWNER task *task,
|
OWNER Task *task,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -107,11 +107,11 @@ void task_table_update(db_handle *db_handle,
|
||||||
* fail_callback.
|
* fail_callback.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void task_table_test_and_update(db_handle *db_handle,
|
void task_table_test_and_update(DBHandle *db_handle,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
int test_state_bitmask,
|
int test_state_bitmask,
|
||||||
int update_state,
|
int update_state,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_get_callback done_callback,
|
task_table_get_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
|
@ -119,15 +119,15 @@ void task_table_test_and_update(db_handle *db_handle,
|
||||||
typedef struct {
|
typedef struct {
|
||||||
int test_state_bitmask;
|
int test_state_bitmask;
|
||||||
int update_state;
|
int update_state;
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
} task_table_test_and_update_data;
|
} TaskTableTestAndUpdateData;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ==== Subscribing to the task table ====
|
* ==== Subscribing to the task table ====
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* Callback for subscribing to the task table. */
|
/* Callback for subscribing to the task table. */
|
||||||
typedef void (*task_table_subscribe_callback)(task *task, void *user_context);
|
typedef void (*task_table_subscribe_callback)(Task *task, void *user_context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Register a callback for a task event. An event is any update of a task in
|
* Register a callback for a task event. An event is any update of a task in
|
||||||
|
@ -152,22 +152,22 @@ typedef void (*task_table_subscribe_callback)(task *task, void *user_context);
|
||||||
* fail_callback.
|
* fail_callback.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void task_table_subscribe(db_handle *db_handle,
|
void task_table_subscribe(DBHandle *db_handle,
|
||||||
db_client_id local_scheduler_id,
|
DBClientID local_scheduler_id,
|
||||||
int state_filter,
|
int state_filter,
|
||||||
task_table_subscribe_callback subscribe_callback,
|
task_table_subscribe_callback subscribe_callback,
|
||||||
void *subscribe_context,
|
void *subscribe_context,
|
||||||
retry_info *retry,
|
RetryInfo *retry,
|
||||||
task_table_done_callback done_callback,
|
task_table_done_callback done_callback,
|
||||||
void *user_context);
|
void *user_context);
|
||||||
|
|
||||||
/* Data that is needed to register task table subscribe callbacks with the state
|
/* Data that is needed to register task table subscribe callbacks with the state
|
||||||
* database. */
|
* database. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
int state_filter;
|
int state_filter;
|
||||||
task_table_subscribe_callback subscribe_callback;
|
task_table_subscribe_callback subscribe_callback;
|
||||||
void *subscribe_context;
|
void *subscribe_context;
|
||||||
} task_table_subscribe_data;
|
} TaskTableSubscribeData;
|
||||||
|
|
||||||
#endif /* task_table_H */
|
#endif /* task_table_H */
|
||||||
|
|
|
@ -24,7 +24,7 @@ typedef struct {
|
||||||
/* Either ARG_BY_REF or ARG_BY_VAL. */
|
/* Either ARG_BY_REF or ARG_BY_VAL. */
|
||||||
int8_t type;
|
int8_t type;
|
||||||
union {
|
union {
|
||||||
object_id obj_id;
|
ObjectID obj_id;
|
||||||
struct {
|
struct {
|
||||||
/* Offset where the data associated to this arg is located relative
|
/* Offset where the data associated to this arg is located relative
|
||||||
* to &task_spec.args_and_returns[0]. */
|
* to &task_spec.args_and_returns[0]. */
|
||||||
|
@ -36,21 +36,21 @@ typedef struct {
|
||||||
|
|
||||||
struct task_spec_impl {
|
struct task_spec_impl {
|
||||||
/** ID of the driver that created this task. */
|
/** ID of the driver that created this task. */
|
||||||
unique_id driver_id;
|
UniqueID driver_id;
|
||||||
/** Task ID of the task. */
|
/** Task ID of the task. */
|
||||||
task_id task_id;
|
TaskID task_id;
|
||||||
/** Task ID of the parent task. */
|
/** Task ID of the parent task. */
|
||||||
task_id parent_task_id;
|
TaskID parent_task_id;
|
||||||
/** A count of the number of tasks submitted by the parent task before this
|
/** A count of the number of tasks submitted by the parent task before this
|
||||||
* one. */
|
* one. */
|
||||||
int64_t parent_counter;
|
int64_t parent_counter;
|
||||||
/** Actor ID of the task. This is the actor that this task is executed on
|
/** 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. */
|
* or NIL_ACTOR_ID if the task is just a normal task. */
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** Number of tasks that have been submitted to this actor so far. */
|
/** Number of tasks that have been submitted to this actor so far. */
|
||||||
int64_t actor_counter;
|
int64_t actor_counter;
|
||||||
/** Function ID of the task. */
|
/** Function ID of the task. */
|
||||||
function_id function_id;
|
FunctionID function_id;
|
||||||
/** Total number of arguments. */
|
/** Total number of arguments. */
|
||||||
int64_t num_args;
|
int64_t num_args;
|
||||||
/** Index of the last argument that has been constructed. */
|
/** Index of the last argument that has been constructed. */
|
||||||
|
@ -78,27 +78,27 @@ struct task_spec_impl {
|
||||||
(sizeof(task_spec) + ((NUM_ARGS) + (NUM_RETURNS)) * sizeof(task_arg) + \
|
(sizeof(task_spec) + ((NUM_ARGS) + (NUM_RETURNS)) * sizeof(task_arg) + \
|
||||||
(ARGS_VALUE_SIZE))
|
(ARGS_VALUE_SIZE))
|
||||||
|
|
||||||
bool task_ids_equal(task_id first_id, task_id second_id) {
|
bool TaskID_equal(TaskID first_id, TaskID second_id) {
|
||||||
return UNIQUE_ID_EQ(first_id, second_id);
|
return UNIQUE_ID_EQ(first_id, second_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool task_id_is_nil(task_id id) {
|
bool TaskID_is_nil(TaskID id) {
|
||||||
return task_ids_equal(id, NIL_TASK_ID);
|
return TaskID_equal(id, NIL_TASK_ID);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool actor_ids_equal(actor_id first_id, actor_id second_id) {
|
bool ActorID_equal(ActorID first_id, ActorID second_id) {
|
||||||
return UNIQUE_ID_EQ(first_id, second_id);
|
return UNIQUE_ID_EQ(first_id, second_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool function_ids_equal(function_id first_id, function_id second_id) {
|
bool FunctionID_equal(FunctionID first_id, FunctionID second_id) {
|
||||||
return UNIQUE_ID_EQ(first_id, second_id);
|
return UNIQUE_ID_EQ(first_id, second_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool function_id_is_nil(function_id id) {
|
bool FunctionID_is_nil(FunctionID id) {
|
||||||
return function_ids_equal(id, NIL_FUNCTION_ID);
|
return FunctionID_equal(id, NIL_FUNCTION_ID);
|
||||||
}
|
}
|
||||||
|
|
||||||
task_id *task_return_ptr(task_spec *spec, int64_t return_index) {
|
TaskID *task_return_ptr(task_spec *spec, int64_t return_index) {
|
||||||
DCHECK(0 <= return_index && return_index < spec->num_returns);
|
DCHECK(0 <= return_index && return_index < spec->num_returns);
|
||||||
task_arg *ret = &spec->args_and_returns[spec->num_args + return_index];
|
task_arg *ret = &spec->args_and_returns[spec->num_args + return_index];
|
||||||
DCHECK(ret->type == ARG_BY_REF);
|
DCHECK(ret->type == ARG_BY_REF);
|
||||||
|
@ -109,12 +109,12 @@ task_id *task_return_ptr(task_spec *spec, int64_t return_index) {
|
||||||
* and that the return IDs have not been set. It assumes the task_spec was
|
* 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
|
* zero-initialized so that uninitialized fields will not make the task ID
|
||||||
* nondeterministic. */
|
* nondeterministic. */
|
||||||
task_id compute_task_id(task_spec *spec) {
|
TaskID compute_task_id(task_spec *spec) {
|
||||||
/* Check that the task ID and return ID fields of the task_spec are
|
/* Check that the task ID and return ID fields of the task_spec are
|
||||||
* uninitialized. */
|
* uninitialized. */
|
||||||
DCHECK(task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
for (int i = 0; i < spec->num_returns; ++i) {
|
for (int i = 0; i < spec->num_returns; ++i) {
|
||||||
DCHECK(object_ids_equal(*task_return_ptr(spec, i), NIL_ID));
|
DCHECK(ObjectID_equal(*task_return_ptr(spec, i), NIL_ID));
|
||||||
}
|
}
|
||||||
/* Compute a SHA256 hash of the task_spec. */
|
/* Compute a SHA256 hash of the task_spec. */
|
||||||
SHA256_CTX ctx;
|
SHA256_CTX ctx;
|
||||||
|
@ -123,18 +123,18 @@ task_id compute_task_id(task_spec *spec) {
|
||||||
sha256_update(&ctx, (BYTE *) spec, task_spec_size(spec));
|
sha256_update(&ctx, (BYTE *) spec, task_spec_size(spec));
|
||||||
sha256_final(&ctx, buff);
|
sha256_final(&ctx, buff);
|
||||||
/* Create a task ID out of the hash. This will truncate the hash. */
|
/* Create a task ID out of the hash. This will truncate the hash. */
|
||||||
task_id task_id;
|
TaskID task_id;
|
||||||
CHECK(sizeof(task_id) <= DIGEST_SIZE);
|
CHECK(sizeof(task_id) <= DIGEST_SIZE);
|
||||||
memcpy(&task_id.id, buff, sizeof(task_id.id));
|
memcpy(&task_id.id, buff, sizeof(task_id.id));
|
||||||
return task_id;
|
return task_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
object_id task_compute_return_id(task_id task_id, int64_t return_index) {
|
ObjectID task_compute_return_id(TaskID task_id, int64_t return_index) {
|
||||||
/* Here, return_indices need to be >= 0, so we can use negative
|
/* Here, return_indices need to be >= 0, so we can use negative
|
||||||
* indices for put. */
|
* indices for put. */
|
||||||
DCHECK(return_index >= 0);
|
DCHECK(return_index >= 0);
|
||||||
/* TODO(rkn): This line requires object and task IDs to be the same size. */
|
/* TODO(rkn): This line requires object and task IDs to be the same size. */
|
||||||
object_id return_id = task_id;
|
ObjectID return_id = task_id;
|
||||||
int64_t *first_bytes = (int64_t *) &return_id;
|
int64_t *first_bytes = (int64_t *) &return_id;
|
||||||
/* XOR the first bytes of the object ID with the return index. We add one so
|
/* XOR the first bytes of the object ID with the return index. We add one so
|
||||||
* the first return ID is not the same as the task ID. */
|
* the first return ID is not the same as the task ID. */
|
||||||
|
@ -142,10 +142,10 @@ object_id task_compute_return_id(task_id task_id, int64_t return_index) {
|
||||||
return return_id;
|
return return_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
object_id task_compute_put_id(task_id task_id, int64_t put_index) {
|
ObjectID task_compute_put_id(TaskID task_id, int64_t put_index) {
|
||||||
DCHECK(put_index >= 0);
|
DCHECK(put_index >= 0);
|
||||||
/* TODO(pcm): This line requires object and task IDs to be the same size. */
|
/* TODO(pcm): This line requires object and task IDs to be the same size. */
|
||||||
object_id put_id = task_id;
|
ObjectID put_id = task_id;
|
||||||
int64_t *first_bytes = (int64_t *) &put_id;
|
int64_t *first_bytes = (int64_t *) &put_id;
|
||||||
/* XOR the first bytes of the object ID with the return index. We add one so
|
/* XOR the first bytes of the object ID with the return index. We add one so
|
||||||
* the first return ID is not the same as the task ID. */
|
* the first return ID is not the same as the task ID. */
|
||||||
|
@ -153,17 +153,17 @@ object_id task_compute_put_id(task_id task_id, int64_t put_index) {
|
||||||
return put_id;
|
return put_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
task_spec *start_construct_task_spec(unique_id driver_id,
|
task_spec *start_construct_task_spec(UniqueID driver_id,
|
||||||
task_id parent_task_id,
|
TaskID parent_task_id,
|
||||||
int64_t parent_counter,
|
int64_t parent_counter,
|
||||||
actor_id actor_id,
|
ActorID actor_id,
|
||||||
int64_t actor_counter,
|
int64_t actor_counter,
|
||||||
function_id function_id,
|
FunctionID function_id,
|
||||||
int64_t num_args,
|
int64_t num_args,
|
||||||
int64_t num_returns,
|
int64_t num_returns,
|
||||||
int64_t args_value_size) {
|
int64_t args_value_size) {
|
||||||
int64_t size = TASK_SPEC_SIZE(num_args, num_returns, args_value_size);
|
int64_t size = TASK_SPEC_SIZE(num_args, num_returns, args_value_size);
|
||||||
task_spec *task = malloc(size);
|
task_spec *task = (task_spec *) malloc(size);
|
||||||
memset(task, 0, size);
|
memset(task, 0, size);
|
||||||
task->driver_id = driver_id;
|
task->driver_id = driver_id;
|
||||||
task->task_id = NIL_TASK_ID;
|
task->task_id = NIL_TASK_ID;
|
||||||
|
@ -197,33 +197,33 @@ int64_t task_spec_size(task_spec *spec) {
|
||||||
spec->args_value_size);
|
spec->args_value_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
function_id task_function(task_spec *spec) {
|
FunctionID task_function(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
return spec->function_id;
|
return spec->function_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
actor_id task_spec_actor_id(task_spec *spec) {
|
ActorID task_spec_actor_id(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
return spec->actor_id;
|
return spec->actor_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
int64_t task_spec_actor_counter(task_spec *spec) {
|
int64_t task_spec_actor_counter(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
return spec->actor_counter;
|
return spec->actor_counter;
|
||||||
}
|
}
|
||||||
|
|
||||||
unique_id task_spec_driver_id(task_spec *spec) {
|
UniqueID task_spec_driver_id(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
return spec->driver_id;
|
return spec->driver_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
task_id task_spec_id(task_spec *spec) {
|
TaskID task_spec_id(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
return spec->task_id;
|
return spec->task_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -240,9 +240,9 @@ int8_t task_arg_type(task_spec *spec, int64_t arg_index) {
|
||||||
return spec->args_and_returns[arg_index].type;
|
return spec->args_and_returns[arg_index].type;
|
||||||
}
|
}
|
||||||
|
|
||||||
object_id task_arg_id(task_spec *spec, int64_t arg_index) {
|
ObjectID task_arg_id(task_spec *spec, int64_t arg_index) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
DCHECK(0 <= arg_index && arg_index < spec->num_args);
|
DCHECK(0 <= arg_index && arg_index < spec->num_args);
|
||||||
task_arg *arg = &spec->args_and_returns[arg_index];
|
task_arg *arg = &spec->args_and_returns[arg_index];
|
||||||
DCHECK(arg->type == ARG_BY_REF)
|
DCHECK(arg->type == ARG_BY_REF)
|
||||||
|
@ -265,9 +265,9 @@ int64_t task_arg_length(task_spec *spec, int64_t arg_index) {
|
||||||
return arg->value.length;
|
return arg->value.length;
|
||||||
}
|
}
|
||||||
|
|
||||||
int64_t task_args_add_ref(task_spec *spec, object_id obj_id) {
|
int64_t task_args_add_ref(task_spec *spec, ObjectID obj_id) {
|
||||||
/* Check that the task is still under construction. */
|
/* Check that the task is still under construction. */
|
||||||
DCHECK(task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
task_arg *arg = &spec->args_and_returns[spec->arg_index];
|
task_arg *arg = &spec->args_and_returns[spec->arg_index];
|
||||||
arg->type = ARG_BY_REF;
|
arg->type = ARG_BY_REF;
|
||||||
arg->obj_id = obj_id;
|
arg->obj_id = obj_id;
|
||||||
|
@ -276,7 +276,7 @@ int64_t task_args_add_ref(task_spec *spec, object_id obj_id) {
|
||||||
|
|
||||||
int64_t task_args_add_val(task_spec *spec, uint8_t *data, int64_t length) {
|
int64_t task_args_add_val(task_spec *spec, uint8_t *data, int64_t length) {
|
||||||
/* Check that the task is still under construction. */
|
/* Check that the task is still under construction. */
|
||||||
DCHECK(task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
task_arg *arg = &spec->args_and_returns[spec->arg_index];
|
task_arg *arg = &spec->args_and_returns[spec->arg_index];
|
||||||
arg->type = ARG_BY_VAL;
|
arg->type = ARG_BY_VAL;
|
||||||
arg->value.offset = spec->args_value_offset;
|
arg->value.offset = spec->args_value_offset;
|
||||||
|
@ -296,9 +296,9 @@ void task_spec_set_required_resource(task_spec *spec,
|
||||||
spec->required_resources[resource_index] = value;
|
spec->required_resources[resource_index] = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
object_id task_return(task_spec *spec, int64_t return_index) {
|
ObjectID task_return(task_spec *spec, int64_t return_index) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
DCHECK(0 <= return_index && return_index < spec->num_returns);
|
DCHECK(0 <= return_index && return_index < spec->num_returns);
|
||||||
task_arg *ret = &spec->args_and_returns[spec->num_args + return_index];
|
task_arg *ret = &spec->args_and_returns[spec->num_args + return_index];
|
||||||
DCHECK(ret->type == ARG_BY_REF);
|
DCHECK(ret->type == ARG_BY_REF);
|
||||||
|
@ -312,7 +312,7 @@ double task_spec_get_required_resource(const task_spec *spec,
|
||||||
|
|
||||||
void free_task_spec(task_spec *spec) {
|
void free_task_spec(task_spec *spec) {
|
||||||
/* Check that the task has been constructed. */
|
/* Check that the task has been constructed. */
|
||||||
DCHECK(!task_ids_equal(spec->task_id, NIL_TASK_ID));
|
DCHECK(!TaskID_equal(spec->task_id, NIL_TASK_ID));
|
||||||
DCHECK(spec->arg_index == spec->num_args);
|
DCHECK(spec->arg_index == spec->num_args);
|
||||||
free(spec);
|
free(spec);
|
||||||
}
|
}
|
||||||
|
@ -322,36 +322,36 @@ void print_task(task_spec *spec, UT_string *output) {
|
||||||
* of bytes compared to the id (+ 1 byte for '\0'). */
|
* of bytes compared to the id (+ 1 byte for '\0'). */
|
||||||
static char hex[ID_STRING_SIZE];
|
static char hex[ID_STRING_SIZE];
|
||||||
/* Print function id. */
|
/* Print function id. */
|
||||||
object_id_to_string((object_id) task_function(spec), &hex[0], ID_STRING_SIZE);
|
ObjectID_to_string((ObjectID) task_function(spec), &hex[0], ID_STRING_SIZE);
|
||||||
utstring_printf(output, "fun %s ", &hex[0]);
|
utstring_printf(output, "fun %s ", &hex[0]);
|
||||||
/* Print arguments. */
|
/* Print arguments. */
|
||||||
for (int i = 0; i < task_num_args(spec); ++i) {
|
for (int i = 0; i < task_num_args(spec); ++i) {
|
||||||
object_id_to_string((object_id) task_arg_id(spec, i), &hex[0],
|
ObjectID_to_string((ObjectID) task_arg_id(spec, i), &hex[0],
|
||||||
ID_STRING_SIZE);
|
ID_STRING_SIZE);
|
||||||
utstring_printf(output, " id:%d %s", i, &hex[0]);
|
utstring_printf(output, " id:%d %s", i, &hex[0]);
|
||||||
}
|
}
|
||||||
/* Print return ids. */
|
/* Print return ids. */
|
||||||
for (int i = 0; i < task_num_returns(spec); ++i) {
|
for (int i = 0; i < task_num_returns(spec); ++i) {
|
||||||
object_id obj_id = task_return(spec, i);
|
ObjectID obj_id = task_return(spec, i);
|
||||||
object_id_to_string(obj_id, &hex[0], ID_STRING_SIZE);
|
ObjectID_to_string(obj_id, &hex[0], ID_STRING_SIZE);
|
||||||
utstring_printf(output, " ret:%d %s", i, &hex[0]);
|
utstring_printf(output, " ret:%d %s", i, &hex[0]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* TASK INSTANCES */
|
/* TASK INSTANCES */
|
||||||
|
|
||||||
struct task_impl {
|
struct TaskImpl {
|
||||||
/** The scheduling state of the task. */
|
/** The scheduling state of the task. */
|
||||||
int state;
|
int state;
|
||||||
/** The ID of the local scheduler involved. */
|
/** The ID of the local scheduler involved. */
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
/** The task specification for this task. */
|
/** The task specification for this task. */
|
||||||
task_spec spec;
|
task_spec spec;
|
||||||
};
|
};
|
||||||
|
|
||||||
task *alloc_task(task_spec *spec, int state, db_client_id local_scheduler_id) {
|
Task *Task_alloc(task_spec *spec, int state, DBClientID local_scheduler_id) {
|
||||||
int64_t size = sizeof(task) - sizeof(task_spec) + task_spec_size(spec);
|
int64_t size = sizeof(Task) - sizeof(task_spec) + task_spec_size(spec);
|
||||||
task *result = malloc(size);
|
Task *result = (Task *) malloc(size);
|
||||||
memset(result, 0, size);
|
memset(result, 0, size);
|
||||||
result->state = state;
|
result->state = state;
|
||||||
result->local_scheduler_id = local_scheduler_id;
|
result->local_scheduler_id = local_scheduler_id;
|
||||||
|
@ -359,43 +359,43 @@ task *alloc_task(task_spec *spec, int state, db_client_id local_scheduler_id) {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
task *copy_task(task *other) {
|
Task *Task_copy(Task *other) {
|
||||||
int64_t size = task_size(other);
|
int64_t size = Task_size(other);
|
||||||
task *copy = malloc(size);
|
Task *copy = (Task *) malloc(size);
|
||||||
CHECK(copy != NULL);
|
CHECK(copy != NULL);
|
||||||
memcpy(copy, other, size);
|
memcpy(copy, other, size);
|
||||||
return copy;
|
return copy;
|
||||||
}
|
}
|
||||||
|
|
||||||
int64_t task_size(task *task_arg) {
|
int64_t Task_size(Task *task_arg) {
|
||||||
return sizeof(task) - sizeof(task_spec) + task_spec_size(&task_arg->spec);
|
return sizeof(Task) - sizeof(task_spec) + task_spec_size(&task_arg->spec);
|
||||||
}
|
}
|
||||||
|
|
||||||
int task_state(task *task) {
|
int Task_state(Task *task) {
|
||||||
return task->state;
|
return task->state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_set_state(task *task, int state) {
|
void Task_set_state(Task *task, int state) {
|
||||||
task->state = state;
|
task->state = state;
|
||||||
}
|
}
|
||||||
|
|
||||||
db_client_id task_local_scheduler(task *task) {
|
DBClientID Task_local_scheduler_id(Task *task) {
|
||||||
return task->local_scheduler_id;
|
return task->local_scheduler_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_set_local_scheduler(task *task, db_client_id local_scheduler_id) {
|
void Task_set_local_scheduler_id(Task *task, DBClientID local_scheduler_id) {
|
||||||
task->local_scheduler_id = local_scheduler_id;
|
task->local_scheduler_id = local_scheduler_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
task_spec *task_task_spec(task *task) {
|
task_spec *Task_task_spec(Task *task) {
|
||||||
return &task->spec;
|
return &task->spec;
|
||||||
}
|
}
|
||||||
|
|
||||||
task_id task_task_id(task *task) {
|
TaskID Task_task_id(Task *task) {
|
||||||
task_spec *spec = task_task_spec(task);
|
task_spec *spec = Task_task_spec(task);
|
||||||
return task_spec_id(spec);
|
return task_spec_id(spec);
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_task(task *task) {
|
void Task_free(Task *task) {
|
||||||
free(task);
|
free(task);
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,19 +18,15 @@
|
||||||
#define NIL_ACTOR_ID NIL_ID
|
#define NIL_ACTOR_ID NIL_ID
|
||||||
#define NIL_FUNCTION_ID NIL_ID
|
#define NIL_FUNCTION_ID NIL_ID
|
||||||
|
|
||||||
typedef unique_id function_id;
|
typedef UniqueID FunctionID;
|
||||||
|
|
||||||
/** The task ID is a deterministic hash of the function ID that the task
|
/** The task ID is a deterministic hash of the function ID that the task
|
||||||
* executes and the argument IDs or argument values. */
|
* executes and the argument IDs or argument values. */
|
||||||
typedef unique_id task_id;
|
typedef UniqueID TaskID;
|
||||||
|
|
||||||
/** The actor ID is the ID of the actor that a task must run on. If the task is
|
/** The actor ID is the ID of the actor that a task must run on. If the task is
|
||||||
* not run on an actor, then NIL_ACTOR_ID should be used. */
|
* not run on an actor, then NIL_ACTOR_ID should be used. */
|
||||||
typedef unique_id actor_id;
|
typedef UniqueID ActorID;
|
||||||
|
|
||||||
/** The task instance ID is a globally unique ID generated which identifies this
|
|
||||||
* particular execution of the task. */
|
|
||||||
typedef unique_id task_iid;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* ==== Task specifications ====
|
* ==== Task specifications ====
|
||||||
|
@ -50,7 +46,7 @@ enum arg_type { ARG_BY_REF, ARG_BY_VAL };
|
||||||
* @param second_id The first task ID to compare.
|
* @param second_id The first task ID to compare.
|
||||||
* @return True if the task IDs are the same and false otherwise.
|
* @return True if the task IDs are the same and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool task_ids_equal(task_id first_id, task_id second_id);
|
bool TaskID_equal(TaskID first_id, TaskID second_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare a task ID to the nil ID.
|
* Compare a task ID to the nil ID.
|
||||||
|
@ -58,7 +54,7 @@ bool task_ids_equal(task_id first_id, task_id second_id);
|
||||||
* @param id The task ID to compare to nil.
|
* @param id The task ID to compare to nil.
|
||||||
* @return True if the task ID is equal to nil.
|
* @return True if the task ID is equal to nil.
|
||||||
*/
|
*/
|
||||||
bool task_id_is_nil(task_id id);
|
bool TaskID_is_nil(TaskID id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare two actor IDs.
|
* Compare two actor IDs.
|
||||||
|
@ -67,7 +63,7 @@ bool task_id_is_nil(task_id id);
|
||||||
* @param second_id The first actor ID to compare.
|
* @param second_id The first actor ID to compare.
|
||||||
* @return True if the actor IDs are the same and false otherwise.
|
* @return True if the actor IDs are the same and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool actor_ids_equal(actor_id first_id, actor_id second_id);
|
bool ActorID_equal(ActorID first_id, ActorID second_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare two function IDs.
|
* Compare two function IDs.
|
||||||
|
@ -76,7 +72,7 @@ bool actor_ids_equal(actor_id first_id, actor_id second_id);
|
||||||
* @param second_id The first function ID to compare.
|
* @param second_id The first function ID to compare.
|
||||||
* @return True if the function IDs are the same and false otherwise.
|
* @return True if the function IDs are the same and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool function_ids_equal(function_id first_id, function_id second_id);
|
bool FunctionID_equal(FunctionID first_id, FunctionID second_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare a function ID to the nil ID.
|
* Compare a function ID to the nil ID.
|
||||||
|
@ -84,7 +80,7 @@ bool function_ids_equal(function_id first_id, function_id second_id);
|
||||||
* @param id The function ID to compare to nil.
|
* @param id The function ID to compare to nil.
|
||||||
* @return True if the function ID is equal to nil.
|
* @return True if the function ID is equal to nil.
|
||||||
*/
|
*/
|
||||||
bool function_id_is_nil(function_id id);
|
bool FunctionID_is_nil(FunctionID id);
|
||||||
|
|
||||||
/* Construct and modify task specifications. */
|
/* Construct and modify task specifications. */
|
||||||
|
|
||||||
|
@ -106,12 +102,12 @@ bool function_id_is_nil(function_id id);
|
||||||
ignoring object ID arguments.
|
ignoring object ID arguments.
|
||||||
* @return The partially constructed task_spec.
|
* @return The partially constructed task_spec.
|
||||||
*/
|
*/
|
||||||
task_spec *start_construct_task_spec(unique_id driver_id,
|
task_spec *start_construct_task_spec(UniqueID driver_id,
|
||||||
task_id parent_task_id,
|
TaskID parent_task_id,
|
||||||
int64_t parent_counter,
|
int64_t parent_counter,
|
||||||
unique_id actor_id,
|
UniqueID actor_id,
|
||||||
int64_t actor_counter,
|
int64_t actor_counter,
|
||||||
function_id function_id,
|
FunctionID function_id,
|
||||||
int64_t num_args,
|
int64_t num_args,
|
||||||
int64_t num_returns,
|
int64_t num_returns,
|
||||||
int64_t args_value_size);
|
int64_t args_value_size);
|
||||||
|
@ -140,7 +136,7 @@ int64_t task_spec_size(task_spec *spec);
|
||||||
* @param spec The task_spec in question.
|
* @param spec The task_spec in question.
|
||||||
* @return The function ID of the function to execute in this task.
|
* @return The function ID of the function to execute in this task.
|
||||||
*/
|
*/
|
||||||
function_id task_function(task_spec *spec);
|
FunctionID task_function(task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the actor ID of the task.
|
* Return the actor ID of the task.
|
||||||
|
@ -148,7 +144,7 @@ function_id task_function(task_spec *spec);
|
||||||
* @param spec The task_spec in question.
|
* @param spec The task_spec in question.
|
||||||
* @return The actor ID of the actor the task is part of.
|
* @return The actor ID of the actor the task is part of.
|
||||||
*/
|
*/
|
||||||
unique_id task_spec_actor_id(task_spec *spec);
|
UniqueID task_spec_actor_id(task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the actor counter of the task. This starts at 0 and increments by 1
|
* Return the actor counter of the task. This starts at 0 and increments by 1
|
||||||
|
@ -165,7 +161,7 @@ int64_t task_spec_actor_counter(task_spec *spec);
|
||||||
* @param spec The task_spec in question.
|
* @param spec The task_spec in question.
|
||||||
* @return The driver ID of the task.
|
* @return The driver ID of the task.
|
||||||
*/
|
*/
|
||||||
unique_id task_spec_driver_id(task_spec *spec);
|
UniqueID task_spec_driver_id(task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the task ID of the task.
|
* Return the task ID of the task.
|
||||||
|
@ -173,7 +169,7 @@ unique_id task_spec_driver_id(task_spec *spec);
|
||||||
* @param spec The task_spec in question.
|
* @param spec The task_spec in question.
|
||||||
* @return The task ID of the task.
|
* @return The task ID of the task.
|
||||||
*/
|
*/
|
||||||
task_id task_spec_id(task_spec *spec);
|
TaskID task_spec_id(task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the number of arguments to this task.
|
* Get the number of arguments to this task.
|
||||||
|
@ -209,7 +205,7 @@ int8_t task_arg_type(task_spec *spec, int64_t arg_index);
|
||||||
* @param arg_index The index of the argument in question.
|
* @param arg_index The index of the argument in question.
|
||||||
* @return The argument at that index.
|
* @return The argument at that index.
|
||||||
*/
|
*/
|
||||||
object_id task_arg_id(task_spec *spec, int64_t arg_index);
|
ObjectID task_arg_id(task_spec *spec, int64_t arg_index);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a particular argument to this task. This assumes the argument is a value.
|
* Get a particular argument to this task. This assumes the argument is a value.
|
||||||
|
@ -239,7 +235,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
|
* @return The number of task arguments that have been set before this one. This
|
||||||
* is only used for testing.
|
* is only used for testing.
|
||||||
*/
|
*/
|
||||||
int64_t task_args_add_ref(task_spec *spec, object_id obj_id);
|
int64_t task_args_add_ref(task_spec *spec, ObjectID obj_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set the next task argument. Note that this API only allows you to set the
|
* Set the next task argument. Note that this API only allows you to set the
|
||||||
|
@ -260,7 +256,7 @@ int64_t task_args_add_val(task_spec *spec, uint8_t *data, int64_t length);
|
||||||
* @param return_index The index of the return object ID in question.
|
* @param return_index The index of the return object ID in question.
|
||||||
* @return The relevant return object ID.
|
* @return The relevant return object ID.
|
||||||
*/
|
*/
|
||||||
object_id task_return(task_spec *spec, int64_t return_index);
|
ObjectID task_return(task_spec *spec, int64_t return_index);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indices into resource vectors.
|
* Indices into resource vectors.
|
||||||
|
@ -308,7 +304,7 @@ double task_spec_get_required_resource(const task_spec *spec,
|
||||||
* @param put_index The number of put calls in this task so far.
|
* @param put_index The number of put calls in this task so far.
|
||||||
* @return The object ID for the object that was put.
|
* @return The object ID for the object that was put.
|
||||||
*/
|
*/
|
||||||
object_id task_compute_put_id(task_id task_id, int64_t put_index);
|
ObjectID task_compute_put_id(TaskID task_id, int64_t put_index);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Free a task_spec.
|
* Free a task_spec.
|
||||||
|
@ -357,7 +353,7 @@ typedef enum {
|
||||||
/** A task is an execution of a task specification. It has a state of execution
|
/** 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
|
* (see scheduling_state) and the ID of the local scheduler it is scheduled on
|
||||||
* or running on. */
|
* or running on. */
|
||||||
typedef struct task_impl task;
|
typedef struct TaskImpl Task;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocate a new task. Must be freed with free_task after use.
|
* Allocate a new task. Must be freed with free_task after use.
|
||||||
|
@ -367,7 +363,7 @@ typedef struct task_impl task;
|
||||||
* @param local_scheduler_id The ID of the local scheduler that the task is
|
* @param local_scheduler_id The ID of the local scheduler that the task is
|
||||||
* scheduled on, if any.
|
* scheduled on, if any.
|
||||||
*/
|
*/
|
||||||
task *alloc_task(task_spec *spec, int state, db_client_id local_scheduler_id);
|
Task *Task_alloc(task_spec *spec, int state, DBClientID local_scheduler_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a copy of the task. Must be freed with free_task after use.
|
* Create a copy of the task. Must be freed with free_task after use.
|
||||||
|
@ -375,30 +371,30 @@ task *alloc_task(task_spec *spec, int state, db_client_id local_scheduler_id);
|
||||||
* @param other The task that will be copied.
|
* @param other The task that will be copied.
|
||||||
* @returns Pointer to the copy of the task.
|
* @returns Pointer to the copy of the task.
|
||||||
*/
|
*/
|
||||||
task *copy_task(task *other);
|
Task *Task_copy(Task *other);
|
||||||
|
|
||||||
/** Size of task structure in bytes. */
|
/** Size of task structure in bytes. */
|
||||||
int64_t task_size(task *task);
|
int64_t Task_size(Task *task);
|
||||||
|
|
||||||
/** The scheduling state of the task. */
|
/** The scheduling state of the task. */
|
||||||
int task_state(task *task);
|
int Task_state(Task *task);
|
||||||
|
|
||||||
/** Update the schedule state of the task. */
|
/** Update the schedule state of the task. */
|
||||||
void task_set_state(task *task, int state);
|
void Task_set_state(Task *task, int state);
|
||||||
|
|
||||||
/** Local scheduler this task has been assigned to or is running on. */
|
/** Local scheduler this task has been assigned to or is running on. */
|
||||||
db_client_id task_local_scheduler(task *task);
|
DBClientID Task_local_scheduler_id(Task *task);
|
||||||
|
|
||||||
/** Set the local scheduler ID for this task. */
|
/** Set the local scheduler ID for this task. */
|
||||||
void task_set_local_scheduler(task *task, db_client_id local_scheduler_id);
|
void Task_set_local_scheduler_id(Task *task, DBClientID local_scheduler_id);
|
||||||
|
|
||||||
/** Task specification of this task. */
|
/** Task specification of this task. */
|
||||||
task_spec *task_task_spec(task *task);
|
task_spec *Task_task_spec(Task *task);
|
||||||
|
|
||||||
/** Task ID of this task. */
|
/** Task ID of this task. */
|
||||||
task_id task_task_id(task *task);
|
TaskID Task_task_id(Task *task);
|
||||||
|
|
||||||
/** Free this task datastructure. */
|
/** Free this task datastructure. */
|
||||||
void free_task(task *task);
|
void Task_free(Task *task);
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
|
@ -6,8 +6,8 @@ SUITE(common_tests);
|
||||||
|
|
||||||
TEST sha1_test(void) {
|
TEST sha1_test(void) {
|
||||||
static char hex[ID_STRING_SIZE];
|
static char hex[ID_STRING_SIZE];
|
||||||
unique_id uid = globally_unique_id();
|
UniqueID uid = globally_unique_id();
|
||||||
object_id_to_string((object_id) uid, &hex[0], ID_STRING_SIZE);
|
ObjectID_to_string((ObjectID) uid, &hex[0], ID_STRING_SIZE);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -34,7 +34,7 @@ const int TEST_NUMBER = 10;
|
||||||
|
|
||||||
/* Test if entries have been written to the database. */
|
/* Test if entries have been written to the database. */
|
||||||
|
|
||||||
void lookup_done_callback(object_id object_id,
|
void lookup_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
|
@ -52,10 +52,10 @@ void lookup_done_callback(object_id object_id,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Entry added to database successfully. */
|
/* Entry added to database successfully. */
|
||||||
void add_done_callback(object_id object_id, void *user_context) {}
|
void add_done_callback(ObjectID object_id, void *user_context) {}
|
||||||
|
|
||||||
/* Test if we got a timeout callback if we couldn't connect database. */
|
/* Test if we got a timeout callback if we couldn't connect database. */
|
||||||
void timeout_callback(object_id object_id, void *context, void *user_data) {
|
void timeout_callback(ObjectID object_id, void *context, void *user_data) {
|
||||||
user_context *uc = (user_context *) context;
|
user_context *uc = (user_context *) context;
|
||||||
CHECK(uc->test_number == TEST_NUMBER)
|
CHECK(uc->test_number == TEST_NUMBER)
|
||||||
}
|
}
|
||||||
|
@ -69,16 +69,16 @@ TEST object_table_lookup_test(void) {
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
/* This uses manager_port1. */
|
/* This uses manager_port1. */
|
||||||
const char *db_connect_args1[] = {"address", "127.0.0.1:12345"};
|
const char *db_connect_args1[] = {"address", "127.0.0.1:12345"};
|
||||||
db_handle *db1 = db_connect("127.0.0.1", 6379, "plasma_manager", manager_addr,
|
DBHandle *db1 = db_connect("127.0.0.1", 6379, "plasma_manager", manager_addr,
|
||||||
2, db_connect_args1);
|
2, db_connect_args1);
|
||||||
/* This uses manager_port2. */
|
/* This uses manager_port2. */
|
||||||
const char *db_connect_args2[] = {"address", "127.0.0.1:12346"};
|
const char *db_connect_args2[] = {"address", "127.0.0.1:12346"};
|
||||||
db_handle *db2 = db_connect("127.0.0.1", 6379, "plasma_manager", manager_addr,
|
DBHandle *db2 = db_connect("127.0.0.1", 6379, "plasma_manager", manager_addr,
|
||||||
2, db_connect_args2);
|
2, db_connect_args2);
|
||||||
db_attach(db1, loop, false);
|
db_attach(db1, loop, false);
|
||||||
db_attach(db2, loop, false);
|
db_attach(db2, loop, false);
|
||||||
unique_id id = globally_unique_id();
|
UniqueID id = globally_unique_id();
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = NUM_RETRIES,
|
.num_retries = NUM_RETRIES,
|
||||||
.timeout = TIMEOUT,
|
.timeout = TIMEOUT,
|
||||||
.fail_callback = timeout_callback,
|
.fail_callback = timeout_callback,
|
||||||
|
@ -109,9 +109,9 @@ TEST object_table_lookup_test(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
int task_table_test_callback_called = 0;
|
int task_table_test_callback_called = 0;
|
||||||
task *task_table_test_task;
|
Task *task_table_test_task;
|
||||||
|
|
||||||
void task_table_test_fail_callback(unique_id id,
|
void task_table_test_fail_callback(UniqueID id,
|
||||||
void *context,
|
void *context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
event_loop *loop = user_data;
|
event_loop *loop = user_data;
|
||||||
|
@ -121,22 +121,22 @@ void task_table_test_fail_callback(unique_id id,
|
||||||
int64_t task_table_delayed_add_task(event_loop *loop,
|
int64_t task_table_delayed_add_task(event_loop *loop,
|
||||||
int64_t id,
|
int64_t id,
|
||||||
void *context) {
|
void *context) {
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = NUM_RETRIES,
|
.num_retries = NUM_RETRIES,
|
||||||
.timeout = TIMEOUT,
|
.timeout = TIMEOUT,
|
||||||
.fail_callback = task_table_test_fail_callback,
|
.fail_callback = task_table_test_fail_callback,
|
||||||
};
|
};
|
||||||
task_table_add_task(db, copy_task(task_table_test_task), &retry, NULL,
|
task_table_add_task(db, Task_copy(task_table_test_task), &retry, NULL,
|
||||||
(void *) loop);
|
(void *) loop);
|
||||||
return EVENT_LOOP_TIMER_DONE;
|
return EVENT_LOOP_TIMER_DONE;
|
||||||
}
|
}
|
||||||
|
|
||||||
void task_table_test_callback(task *callback_task, void *user_data) {
|
void task_table_test_callback(Task *callback_task, void *user_data) {
|
||||||
task_table_test_callback_called = 1;
|
task_table_test_callback_called = 1;
|
||||||
CHECK(task_state(callback_task) == TASK_STATUS_SCHEDULED);
|
CHECK(Task_state(callback_task) == TASK_STATUS_SCHEDULED);
|
||||||
CHECK(task_size(callback_task) == task_size(task_table_test_task));
|
CHECK(Task_size(callback_task) == Task_size(task_table_test_task));
|
||||||
CHECK(memcmp(callback_task, task_table_test_task, task_size(callback_task)) ==
|
CHECK(memcmp(callback_task, task_table_test_task, Task_size(callback_task)) ==
|
||||||
0);
|
0);
|
||||||
event_loop *loop = user_data;
|
event_loop *loop = user_data;
|
||||||
event_loop_stop(loop);
|
event_loop_stop(loop);
|
||||||
|
@ -145,15 +145,15 @@ void task_table_test_callback(task *callback_task, void *user_data) {
|
||||||
TEST task_table_test(void) {
|
TEST task_table_test(void) {
|
||||||
task_table_test_callback_called = 0;
|
task_table_test_callback_called = 0;
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, loop, false);
|
db_attach(db, loop, false);
|
||||||
db_client_id local_scheduler_id = globally_unique_id();
|
DBClientID local_scheduler_id = globally_unique_id();
|
||||||
task_spec *spec = example_task_spec(1, 1);
|
task_spec *spec = example_task_spec(1, 1);
|
||||||
task_table_test_task =
|
task_table_test_task =
|
||||||
alloc_task(spec, TASK_STATUS_SCHEDULED, local_scheduler_id);
|
Task_alloc(spec, TASK_STATUS_SCHEDULED, local_scheduler_id);
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = NUM_RETRIES,
|
.num_retries = NUM_RETRIES,
|
||||||
.timeout = TIMEOUT,
|
.timeout = TIMEOUT,
|
||||||
.fail_callback = task_table_test_fail_callback,
|
.fail_callback = task_table_test_fail_callback,
|
||||||
|
@ -164,7 +164,7 @@ TEST task_table_test(void) {
|
||||||
event_loop_add_timer(
|
event_loop_add_timer(
|
||||||
loop, 200, (event_loop_timer_handler) task_table_delayed_add_task, db);
|
loop, 200, (event_loop_timer_handler) task_table_delayed_add_task, db);
|
||||||
event_loop_run(loop);
|
event_loop_run(loop);
|
||||||
free_task(task_table_test_task);
|
Task_free(task_table_test_task);
|
||||||
db_disconnect(db);
|
db_disconnect(db);
|
||||||
destroy_outstanding_callbacks(loop);
|
destroy_outstanding_callbacks(loop);
|
||||||
event_loop_destroy(loop);
|
event_loop_destroy(loop);
|
||||||
|
@ -174,20 +174,20 @@ TEST task_table_test(void) {
|
||||||
|
|
||||||
int num_test_callback_called = 0;
|
int num_test_callback_called = 0;
|
||||||
|
|
||||||
void task_table_all_test_callback(task *task, void *user_data) {
|
void task_table_all_test_callback(Task *task, void *user_data) {
|
||||||
num_test_callback_called += 1;
|
num_test_callback_called += 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST task_table_all_test(void) {
|
TEST task_table_all_test(void) {
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "local_scheduler", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, loop, false);
|
db_attach(db, loop, false);
|
||||||
task_spec *spec = example_task_spec(1, 1);
|
task_spec *spec = example_task_spec(1, 1);
|
||||||
/* Schedule two tasks on different local local schedulers. */
|
/* Schedule two tasks on different local local schedulers. */
|
||||||
task *task1 = alloc_task(spec, TASK_STATUS_SCHEDULED, globally_unique_id());
|
Task *task1 = Task_alloc(spec, TASK_STATUS_SCHEDULED, globally_unique_id());
|
||||||
task *task2 = alloc_task(spec, TASK_STATUS_SCHEDULED, globally_unique_id());
|
Task *task2 = Task_alloc(spec, TASK_STATUS_SCHEDULED, globally_unique_id());
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = NUM_RETRIES, .timeout = TIMEOUT, .fail_callback = NULL,
|
.num_retries = NUM_RETRIES, .timeout = TIMEOUT, .fail_callback = NULL,
|
||||||
};
|
};
|
||||||
task_table_subscribe(db, NIL_ID, TASK_STATUS_SCHEDULED,
|
task_table_subscribe(db, NIL_ID, TASK_STATUS_SCHEDULED,
|
||||||
|
@ -212,8 +212,8 @@ TEST task_table_all_test(void) {
|
||||||
TEST unique_client_id_test(void) {
|
TEST unique_client_id_test(void) {
|
||||||
enum { num_conns = 100 };
|
enum { num_conns = 100 };
|
||||||
|
|
||||||
db_client_id ids[num_conns];
|
DBClientID ids[num_conns];
|
||||||
db_handle *db;
|
DBHandle *db;
|
||||||
for (int i = 0; i < num_conns; ++i) {
|
for (int i = 0; i < num_conns; ++i) {
|
||||||
db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
ids[i] = get_db_client_id(db);
|
ids[i] = get_db_client_id(db);
|
||||||
|
@ -221,7 +221,7 @@ TEST unique_client_id_test(void) {
|
||||||
}
|
}
|
||||||
for (int i = 0; i < num_conns; ++i) {
|
for (int i = 0; i < num_conns; ++i) {
|
||||||
for (int j = 0; j < i; ++j) {
|
for (int j = 0; j < i; ++j) {
|
||||||
ASSERT(!db_client_ids_equal(ids[i], ids[j]));
|
ASSERT(!DBClientID_equal(ids[i], ids[j]));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
PASS();
|
PASS();
|
||||||
|
|
|
@ -16,12 +16,12 @@ static event_loop *g_loop;
|
||||||
|
|
||||||
int new_object_failed = 0;
|
int new_object_failed = 0;
|
||||||
int new_object_succeeded = 0;
|
int new_object_succeeded = 0;
|
||||||
object_id new_object_id;
|
ObjectID new_object_id;
|
||||||
task *new_object_task;
|
Task *new_object_task;
|
||||||
task_spec *new_object_task_spec;
|
task_spec *new_object_task_spec;
|
||||||
task_id new_object_task_id;
|
TaskID new_object_task_id;
|
||||||
|
|
||||||
void new_object_fail_callback(unique_id id,
|
void new_object_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
new_object_failed = 1;
|
new_object_failed = 1;
|
||||||
|
@ -30,34 +30,34 @@ void new_object_fail_callback(unique_id id,
|
||||||
|
|
||||||
/* === Test adding an object with an associated task === */
|
/* === Test adding an object with an associated task === */
|
||||||
|
|
||||||
void new_object_done_callback(object_id object_id,
|
void new_object_done_callback(ObjectID object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
new_object_succeeded = 1;
|
new_object_succeeded = 1;
|
||||||
CHECK(object_ids_equal(object_id, new_object_id));
|
CHECK(ObjectID_equal(object_id, new_object_id));
|
||||||
CHECK(task_ids_equal(task_id, new_object_task_id));
|
CHECK(TaskID_equal(task_id, new_object_task_id));
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
}
|
}
|
||||||
|
|
||||||
void new_object_lookup_callback(object_id object_id, void *user_context) {
|
void new_object_lookup_callback(ObjectID object_id, void *user_context) {
|
||||||
CHECK(object_ids_equal(object_id, new_object_id));
|
CHECK(ObjectID_equal(object_id, new_object_id));
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = new_object_fail_callback,
|
.fail_callback = new_object_fail_callback,
|
||||||
};
|
};
|
||||||
db_handle *db = user_context;
|
DBHandle *db = user_context;
|
||||||
result_table_lookup(db, new_object_id, &retry, new_object_done_callback,
|
result_table_lookup(db, new_object_id, &retry, new_object_done_callback,
|
||||||
NULL);
|
NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
void new_object_task_callback(task_id task_id, void *user_context) {
|
void new_object_task_callback(TaskID task_id, void *user_context) {
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = new_object_fail_callback,
|
.fail_callback = new_object_fail_callback,
|
||||||
};
|
};
|
||||||
db_handle *db = user_context;
|
DBHandle *db = user_context;
|
||||||
result_table_add(db, new_object_id, new_object_task_id, &retry,
|
result_table_add(db, new_object_id, new_object_task_id, &retry,
|
||||||
new_object_lookup_callback, (void *) db);
|
new_object_lookup_callback, (void *) db);
|
||||||
}
|
}
|
||||||
|
@ -67,18 +67,18 @@ TEST new_object_test(void) {
|
||||||
new_object_succeeded = 0;
|
new_object_succeeded = 0;
|
||||||
new_object_id = globally_unique_id();
|
new_object_id = globally_unique_id();
|
||||||
new_object_task = example_task(1, 1, TASK_STATUS_WAITING);
|
new_object_task = example_task(1, 1, TASK_STATUS_WAITING);
|
||||||
new_object_task_spec = task_task_spec(new_object_task);
|
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 = task_spec_id(new_object_task_spec);
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = new_object_fail_callback,
|
.fail_callback = new_object_fail_callback,
|
||||||
};
|
};
|
||||||
task_table_add_task(db, copy_task(new_object_task), &retry,
|
task_table_add_task(db, Task_copy(new_object_task), &retry,
|
||||||
new_object_task_callback, db);
|
new_object_task_callback, db);
|
||||||
event_loop_run(g_loop);
|
event_loop_run(g_loop);
|
||||||
db_disconnect(db);
|
db_disconnect(db);
|
||||||
|
@ -91,8 +91,8 @@ TEST new_object_test(void) {
|
||||||
|
|
||||||
/* === Test adding an object without an associated task === */
|
/* === Test adding an object without an associated task === */
|
||||||
|
|
||||||
void new_object_no_task_callback(object_id object_id,
|
void new_object_no_task_callback(ObjectID object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
new_object_succeeded = 1;
|
new_object_succeeded = 1;
|
||||||
CHECK(IS_NIL_ID(task_id));
|
CHECK(IS_NIL_ID(task_id));
|
||||||
|
@ -105,10 +105,10 @@ TEST new_object_no_task_test(void) {
|
||||||
new_object_id = globally_unique_id();
|
new_object_id = globally_unique_id();
|
||||||
new_object_task_id = globally_unique_id();
|
new_object_task_id = globally_unique_id();
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = new_object_fail_callback,
|
.fail_callback = new_object_fail_callback,
|
||||||
|
@ -131,7 +131,7 @@ TEST new_object_no_task_test(void) {
|
||||||
const char *lookup_timeout_context = "lookup_timeout";
|
const char *lookup_timeout_context = "lookup_timeout";
|
||||||
int lookup_failed = 0;
|
int lookup_failed = 0;
|
||||||
|
|
||||||
void lookup_done_callback(object_id object_id,
|
void lookup_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -139,7 +139,7 @@ void lookup_done_callback(object_id object_id,
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void lookup_fail_callback(unique_id id, void *user_context, void *user_data) {
|
void lookup_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
lookup_failed = 1;
|
lookup_failed = 1;
|
||||||
CHECK(user_context == (void *) lookup_timeout_context);
|
CHECK(user_context == (void *) lookup_timeout_context);
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
|
@ -147,10 +147,10 @@ void lookup_fail_callback(unique_id id, void *user_context, void *user_data) {
|
||||||
|
|
||||||
TEST lookup_timeout_test(void) {
|
TEST lookup_timeout_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5, .timeout = 100, .fail_callback = lookup_fail_callback,
|
.num_retries = 5, .timeout = 100, .fail_callback = lookup_fail_callback,
|
||||||
};
|
};
|
||||||
object_table_lookup(db, NIL_ID, &retry, lookup_done_callback,
|
object_table_lookup(db, NIL_ID, &retry, lookup_done_callback,
|
||||||
|
@ -170,12 +170,12 @@ TEST lookup_timeout_test(void) {
|
||||||
const char *add_timeout_context = "add_timeout";
|
const char *add_timeout_context = "add_timeout";
|
||||||
int add_failed = 0;
|
int add_failed = 0;
|
||||||
|
|
||||||
void add_done_callback(object_id object_id, void *user_context) {
|
void add_done_callback(ObjectID object_id, void *user_context) {
|
||||||
/* The done callback should not be called. */
|
/* The done callback should not be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_fail_callback(unique_id id, void *user_context, void *user_data) {
|
void add_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
add_failed = 1;
|
add_failed = 1;
|
||||||
CHECK(user_context == (void *) add_timeout_context);
|
CHECK(user_context == (void *) add_timeout_context);
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
|
@ -183,10 +183,10 @@ void add_fail_callback(unique_id id, void *user_context, void *user_data) {
|
||||||
|
|
||||||
TEST add_timeout_test(void) {
|
TEST add_timeout_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5, .timeout = 100, .fail_callback = add_fail_callback,
|
.num_retries = 5, .timeout = 100, .fail_callback = add_fail_callback,
|
||||||
};
|
};
|
||||||
object_table_add(db, NIL_ID, 0, (unsigned char *) NIL_DIGEST, &retry,
|
object_table_add(db, NIL_ID, 0, (unsigned char *) NIL_DIGEST, &retry,
|
||||||
|
@ -205,7 +205,7 @@ TEST add_timeout_test(void) {
|
||||||
|
|
||||||
int subscribe_failed = 0;
|
int subscribe_failed = 0;
|
||||||
|
|
||||||
void subscribe_done_callback(object_id object_id,
|
void subscribe_done_callback(ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
|
@ -214,19 +214,17 @@ void subscribe_done_callback(object_id object_id,
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_fail_callback(unique_id id,
|
void subscribe_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
void *user_context,
|
|
||||||
void *user_data) {
|
|
||||||
subscribe_failed = 1;
|
subscribe_failed = 1;
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST subscribe_timeout_test(void) {
|
TEST subscribe_timeout_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = subscribe_fail_callback,
|
.fail_callback = subscribe_fail_callback,
|
||||||
|
@ -248,7 +246,7 @@ TEST subscribe_timeout_test(void) {
|
||||||
int64_t reconnect_context_callback(event_loop *loop,
|
int64_t reconnect_context_callback(event_loop *loop,
|
||||||
int64_t timer_id,
|
int64_t timer_id,
|
||||||
void *context) {
|
void *context) {
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
/* Reconnect to redis. This is not reconnecting the pub/sub channel. */
|
/* Reconnect to redis. This is not reconnecting the pub/sub channel. */
|
||||||
redisAsyncFree(db->context);
|
redisAsyncFree(db->context);
|
||||||
redisFree(db->sync_context);
|
redisFree(db->sync_context);
|
||||||
|
@ -273,7 +271,7 @@ int64_t terminate_event_loop_callback(event_loop *loop,
|
||||||
const char *lookup_retry_context = "lookup_retry";
|
const char *lookup_retry_context = "lookup_retry";
|
||||||
int lookup_retry_succeeded = 0;
|
int lookup_retry_succeeded = 0;
|
||||||
|
|
||||||
void lookup_retry_done_callback(object_id object_id,
|
void lookup_retry_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -281,7 +279,7 @@ void lookup_retry_done_callback(object_id object_id,
|
||||||
lookup_retry_succeeded = 1;
|
lookup_retry_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void lookup_retry_fail_callback(unique_id id,
|
void lookup_retry_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* The fail callback should not be called. */
|
/* The fail callback should not be called. */
|
||||||
|
@ -295,7 +293,7 @@ int add_retry_succeeded = 0;
|
||||||
|
|
||||||
/* === Test add then lookup retry === */
|
/* === Test add then lookup retry === */
|
||||||
|
|
||||||
void add_lookup_done_callback(object_id object_id,
|
void add_lookup_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -305,9 +303,9 @@ void add_lookup_done_callback(object_id object_id,
|
||||||
lookup_retry_succeeded = 1;
|
lookup_retry_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_lookup_callback(object_id object_id, void *user_context) {
|
void add_lookup_callback(ObjectID object_id, void *user_context) {
|
||||||
db_handle *db = user_context;
|
DBHandle *db = user_context;
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = lookup_retry_fail_callback,
|
.fail_callback = lookup_retry_fail_callback,
|
||||||
|
@ -321,10 +319,10 @@ TEST add_lookup_test(void) {
|
||||||
lookup_retry_succeeded = 0;
|
lookup_retry_succeeded = 0;
|
||||||
/* Construct the arguments to db_connect. */
|
/* Construct the arguments to db_connect. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:11235"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:11235"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 2,
|
||||||
2, db_connect_args);
|
db_connect_args);
|
||||||
db_attach(db, g_loop, true);
|
db_attach(db, g_loop, true);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = lookup_retry_fail_callback,
|
.fail_callback = lookup_retry_fail_callback,
|
||||||
|
@ -344,7 +342,7 @@ TEST add_lookup_test(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* === Test add, remove, then lookup === */
|
/* === Test add, remove, then lookup === */
|
||||||
void add_remove_lookup_done_callback(object_id object_id,
|
void add_remove_lookup_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -353,9 +351,9 @@ void add_remove_lookup_done_callback(object_id object_id,
|
||||||
lookup_retry_succeeded = 1;
|
lookup_retry_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_remove_lookup_callback(object_id object_id, void *user_context) {
|
void add_remove_lookup_callback(ObjectID object_id, void *user_context) {
|
||||||
db_handle *db = user_context;
|
DBHandle *db = user_context;
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = lookup_retry_fail_callback,
|
.fail_callback = lookup_retry_fail_callback,
|
||||||
|
@ -364,9 +362,9 @@ void add_remove_lookup_callback(object_id object_id, void *user_context) {
|
||||||
(void *) lookup_retry_context);
|
(void *) lookup_retry_context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_remove_callback(object_id object_id, void *user_context) {
|
void add_remove_callback(ObjectID object_id, void *user_context) {
|
||||||
db_handle *db = user_context;
|
DBHandle *db = user_context;
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = lookup_retry_fail_callback,
|
.fail_callback = lookup_retry_fail_callback,
|
||||||
|
@ -378,10 +376,10 @@ void add_remove_callback(object_id object_id, void *user_context) {
|
||||||
TEST add_remove_lookup_test(void) {
|
TEST add_remove_lookup_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
lookup_retry_succeeded = 0;
|
lookup_retry_succeeded = 0;
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, true);
|
db_attach(db, g_loop, true);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = lookup_retry_fail_callback,
|
.fail_callback = lookup_retry_fail_callback,
|
||||||
|
@ -408,7 +406,7 @@ int subscribe_retry_succeeded = 0;
|
||||||
int64_t reconnect_sub_context_callback(event_loop *loop,
|
int64_t reconnect_sub_context_callback(event_loop *loop,
|
||||||
int64_t timer_id,
|
int64_t timer_id,
|
||||||
void *context) {
|
void *context) {
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
/* Reconnect to redis. This is not reconnecting the pub/sub channel. */
|
/* Reconnect to redis. This is not reconnecting the pub/sub channel. */
|
||||||
redisAsyncFree(db->sub_context);
|
redisAsyncFree(db->sub_context);
|
||||||
redisAsyncFree(db->context);
|
redisAsyncFree(db->context);
|
||||||
|
@ -430,14 +428,14 @@ int64_t reconnect_sub_context_callback(event_loop *loop,
|
||||||
const char *lookup_late_context = "lookup_late";
|
const char *lookup_late_context = "lookup_late";
|
||||||
int lookup_late_failed = 0;
|
int lookup_late_failed = 0;
|
||||||
|
|
||||||
void lookup_late_fail_callback(unique_id id,
|
void lookup_late_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
CHECK(user_context == (void *) lookup_late_context);
|
CHECK(user_context == (void *) lookup_late_context);
|
||||||
lookup_late_failed = 1;
|
lookup_late_failed = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void lookup_late_done_callback(object_id object_id,
|
void lookup_late_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -447,10 +445,10 @@ void lookup_late_done_callback(object_id object_id,
|
||||||
|
|
||||||
TEST lookup_late_test(void) {
|
TEST lookup_late_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0,
|
.num_retries = 0,
|
||||||
.timeout = 0,
|
.timeout = 0,
|
||||||
.fail_callback = lookup_late_fail_callback,
|
.fail_callback = lookup_late_fail_callback,
|
||||||
|
@ -477,22 +475,22 @@ TEST lookup_late_test(void) {
|
||||||
const char *add_late_context = "add_late";
|
const char *add_late_context = "add_late";
|
||||||
int add_late_failed = 0;
|
int add_late_failed = 0;
|
||||||
|
|
||||||
void add_late_fail_callback(unique_id id, void *user_context, void *user_data) {
|
void add_late_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
CHECK(user_context == (void *) add_late_context);
|
CHECK(user_context == (void *) add_late_context);
|
||||||
add_late_failed = 1;
|
add_late_failed = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_late_done_callback(object_id object_id, void *user_context) {
|
void add_late_done_callback(ObjectID object_id, void *user_context) {
|
||||||
/* This function should never be called. */
|
/* This function should never be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST add_late_test(void) {
|
TEST add_late_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 0, .fail_callback = add_late_fail_callback,
|
.num_retries = 0, .timeout = 0, .fail_callback = add_late_fail_callback,
|
||||||
};
|
};
|
||||||
object_table_add(db, NIL_ID, 0, (unsigned char *) NIL_DIGEST, &retry,
|
object_table_add(db, NIL_ID, 0, (unsigned char *) NIL_DIGEST, &retry,
|
||||||
|
@ -517,14 +515,14 @@ TEST add_late_test(void) {
|
||||||
const char *subscribe_late_context = "subscribe_late";
|
const char *subscribe_late_context = "subscribe_late";
|
||||||
int subscribe_late_failed = 0;
|
int subscribe_late_failed = 0;
|
||||||
|
|
||||||
void subscribe_late_fail_callback(unique_id id,
|
void subscribe_late_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
CHECK(user_context == (void *) subscribe_late_context);
|
CHECK(user_context == (void *) subscribe_late_context);
|
||||||
subscribe_late_failed = 1;
|
subscribe_late_failed = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_late_done_callback(object_id object_id,
|
void subscribe_late_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
|
@ -534,10 +532,10 @@ void subscribe_late_done_callback(object_id object_id,
|
||||||
|
|
||||||
TEST subscribe_late_test(void) {
|
TEST subscribe_late_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0,
|
.num_retries = 0,
|
||||||
.timeout = 0,
|
.timeout = 0,
|
||||||
.fail_callback = subscribe_late_fail_callback,
|
.fail_callback = subscribe_late_fail_callback,
|
||||||
|
@ -565,34 +563,34 @@ TEST subscribe_late_test(void) {
|
||||||
const char *subscribe_success_context = "subscribe_success";
|
const char *subscribe_success_context = "subscribe_success";
|
||||||
int subscribe_success_done = 0;
|
int subscribe_success_done = 0;
|
||||||
int subscribe_success_succeeded = 0;
|
int subscribe_success_succeeded = 0;
|
||||||
object_id subscribe_id;
|
ObjectID subscribe_id;
|
||||||
|
|
||||||
void subscribe_success_fail_callback(unique_id id,
|
void subscribe_success_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* This function should never be called. */
|
/* This function should never be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_success_done_callback(object_id object_id,
|
void subscribe_success_done_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 750, .fail_callback = NULL,
|
.num_retries = 0, .timeout = 750, .fail_callback = NULL,
|
||||||
};
|
};
|
||||||
object_table_add((db_handle *) user_context, subscribe_id, 0,
|
object_table_add((DBHandle *) user_context, subscribe_id, 0,
|
||||||
(unsigned char *) NIL_DIGEST, &retry, NULL, NULL);
|
(unsigned char *) NIL_DIGEST, &retry, NULL, NULL);
|
||||||
subscribe_success_done = 1;
|
subscribe_success_done = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_success_object_available_callback(object_id object_id,
|
void subscribe_success_object_available_callback(ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
CHECK(user_context == (void *) subscribe_success_context);
|
CHECK(user_context == (void *) subscribe_success_context);
|
||||||
CHECK(object_ids_equal(object_id, subscribe_id));
|
CHECK(ObjectID_equal(object_id, subscribe_id));
|
||||||
CHECK(manager_count == 1);
|
CHECK(manager_count == 1);
|
||||||
subscribe_success_succeeded = 1;
|
subscribe_success_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
@ -602,12 +600,12 @@ TEST subscribe_success_test(void) {
|
||||||
|
|
||||||
/* Construct the arguments to db_connect. */
|
/* Construct the arguments to db_connect. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 2,
|
||||||
2, db_connect_args);
|
db_connect_args);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
subscribe_id = globally_unique_id();
|
subscribe_id = globally_unique_id();
|
||||||
|
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0,
|
.num_retries = 0,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = subscribe_success_fail_callback,
|
.fail_callback = subscribe_success_fail_callback,
|
||||||
|
@ -617,7 +615,7 @@ TEST subscribe_success_test(void) {
|
||||||
(void *) subscribe_success_context, &retry,
|
(void *) subscribe_success_context, &retry,
|
||||||
subscribe_success_done_callback, (void *) db);
|
subscribe_success_done_callback, (void *) db);
|
||||||
|
|
||||||
object_id object_ids[1] = {subscribe_id};
|
ObjectID object_ids[1] = {subscribe_id};
|
||||||
object_table_request_notifications(db, 1, object_ids, &retry);
|
object_table_request_notifications(db, 1, object_ids, &retry);
|
||||||
|
|
||||||
/* Install handler for terminating the event loop. */
|
/* Install handler for terminating the event loop. */
|
||||||
|
@ -645,7 +643,7 @@ const char *subscribe_object_present_str = "subscribe_object_present";
|
||||||
int subscribe_object_present_succeeded = 0;
|
int subscribe_object_present_succeeded = 0;
|
||||||
|
|
||||||
void subscribe_object_present_object_available_callback(
|
void subscribe_object_present_object_available_callback(
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
|
@ -658,7 +656,7 @@ void subscribe_object_present_object_available_callback(
|
||||||
CHECK(manager_count == 1);
|
CHECK(manager_count == 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
void fatal_fail_callback(unique_id id, void *user_context, void *user_data) {
|
void fatal_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
/* This function should never be called. */
|
/* This function should never be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
@ -671,11 +669,11 @@ TEST subscribe_object_present_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
/* Construct the arguments to db_connect. */
|
/* Construct the arguments to db_connect. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 2,
|
||||||
2, db_connect_args);
|
db_connect_args);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
unique_id id = globally_unique_id();
|
UniqueID id = globally_unique_id();
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 100, .fail_callback = fatal_fail_callback,
|
.num_retries = 0, .timeout = 100, .fail_callback = fatal_fail_callback,
|
||||||
};
|
};
|
||||||
object_table_add(db, id, data_size, (unsigned char *) NIL_DIGEST, &retry,
|
object_table_add(db, id, data_size, (unsigned char *) NIL_DIGEST, &retry,
|
||||||
|
@ -690,7 +688,7 @@ TEST subscribe_object_present_test(void) {
|
||||||
/* Run the event loop to create do the add and subscribe. */
|
/* Run the event loop to create do the add and subscribe. */
|
||||||
event_loop_run(g_loop);
|
event_loop_run(g_loop);
|
||||||
|
|
||||||
object_id object_ids[1] = {id};
|
ObjectID object_ids[1] = {id};
|
||||||
object_table_request_notifications(db, 1, object_ids, &retry);
|
object_table_request_notifications(db, 1, object_ids, &retry);
|
||||||
/* Install handler for terminating the event loop. */
|
/* Install handler for terminating the event loop. */
|
||||||
event_loop_add_timer(g_loop, 750,
|
event_loop_add_timer(g_loop, 750,
|
||||||
|
@ -712,7 +710,7 @@ const char *subscribe_object_not_present_context =
|
||||||
"subscribe_object_not_present";
|
"subscribe_object_not_present";
|
||||||
|
|
||||||
void subscribe_object_not_present_object_available_callback(
|
void subscribe_object_not_present_object_available_callback(
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
|
@ -723,11 +721,11 @@ void subscribe_object_not_present_object_available_callback(
|
||||||
|
|
||||||
TEST subscribe_object_not_present_test(void) {
|
TEST subscribe_object_not_present_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
unique_id id = globally_unique_id();
|
UniqueID id = globally_unique_id();
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
||||||
};
|
};
|
||||||
object_table_subscribe_to_notifications(
|
object_table_subscribe_to_notifications(
|
||||||
|
@ -740,7 +738,7 @@ TEST subscribe_object_not_present_test(void) {
|
||||||
/* Run the event loop to do the subscribe. */
|
/* Run the event loop to do the subscribe. */
|
||||||
event_loop_run(g_loop);
|
event_loop_run(g_loop);
|
||||||
|
|
||||||
object_id object_ids[1] = {id};
|
ObjectID object_ids[1] = {id};
|
||||||
object_table_request_notifications(db, 1, object_ids, &retry);
|
object_table_request_notifications(db, 1, object_ids, &retry);
|
||||||
/* Install handler for terminating the event loop. */
|
/* Install handler for terminating the event loop. */
|
||||||
event_loop_add_timer(g_loop, 750,
|
event_loop_add_timer(g_loop, 750,
|
||||||
|
@ -762,7 +760,7 @@ const char *subscribe_object_available_later_context =
|
||||||
int subscribe_object_available_later_succeeded = 0;
|
int subscribe_object_available_later_succeeded = 0;
|
||||||
|
|
||||||
void subscribe_object_available_later_object_available_callback(
|
void subscribe_object_available_later_object_available_callback(
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
|
@ -786,11 +784,11 @@ TEST subscribe_object_available_later_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
/* Construct the arguments to db_connect. */
|
/* Construct the arguments to db_connect. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 2,
|
||||||
2, db_connect_args);
|
db_connect_args);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
unique_id id = globally_unique_id();
|
UniqueID id = globally_unique_id();
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
||||||
};
|
};
|
||||||
object_table_subscribe_to_notifications(
|
object_table_subscribe_to_notifications(
|
||||||
|
@ -803,7 +801,7 @@ TEST subscribe_object_available_later_test(void) {
|
||||||
/* Run the event loop to do the subscribe. */
|
/* Run the event loop to do the subscribe. */
|
||||||
event_loop_run(g_loop);
|
event_loop_run(g_loop);
|
||||||
|
|
||||||
object_id object_ids[1] = {id};
|
ObjectID object_ids[1] = {id};
|
||||||
object_table_request_notifications(db, 1, object_ids, &retry);
|
object_table_request_notifications(db, 1, object_ids, &retry);
|
||||||
/* Install handler for terminating the event loop. */
|
/* Install handler for terminating the event loop. */
|
||||||
event_loop_add_timer(g_loop, 750,
|
event_loop_add_timer(g_loop, 750,
|
||||||
|
@ -839,11 +837,11 @@ TEST subscribe_object_available_subscribe_all(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
/* Construct the arguments to db_connect. */
|
/* Construct the arguments to db_connect. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:11236"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 2,
|
||||||
2, db_connect_args);
|
db_connect_args);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
unique_id id = globally_unique_id();
|
UniqueID id = globally_unique_id();
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
.num_retries = 0, .timeout = 100, .fail_callback = NULL,
|
||||||
};
|
};
|
||||||
object_table_subscribe_to_notifications(
|
object_table_subscribe_to_notifications(
|
||||||
|
|
|
@ -69,7 +69,7 @@ TEST redis_socket_test(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void redis_read_callback(event_loop *loop, int fd, void *context, int events) {
|
void redis_read_callback(event_loop *loop, int fd, void *context, int events) {
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
char *cmd = read_log_message(fd);
|
char *cmd = read_log_message(fd);
|
||||||
redisAsyncCommand(db->context, async_redis_socket_test_callback, NULL, cmd);
|
redisAsyncCommand(db->context, async_redis_socket_test_callback, NULL, cmd);
|
||||||
free(cmd);
|
free(cmd);
|
||||||
|
@ -102,7 +102,7 @@ TEST async_redis_socket_test(void) {
|
||||||
utarray_push_back(connections, &socket_fd);
|
utarray_push_back(connections, &socket_fd);
|
||||||
|
|
||||||
/* Start connection to Redis. */
|
/* Start connection to Redis. */
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "test_process", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "test_process", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, loop, false);
|
db_attach(db, loop, false);
|
||||||
|
|
||||||
|
@ -148,7 +148,7 @@ void logging_read_callback(event_loop *loop,
|
||||||
int fd,
|
int fd,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
db_handle *conn = context;
|
DBHandle *conn = context;
|
||||||
char *cmd = read_log_message(fd);
|
char *cmd = read_log_message(fd);
|
||||||
redisAsyncCommand(conn->context, logging_test_callback, NULL, cmd,
|
redisAsyncCommand(conn->context, logging_test_callback, NULL, cmd,
|
||||||
(char *) conn->client.id, sizeof(conn->client.id));
|
(char *) conn->client.id, sizeof(conn->client.id));
|
||||||
|
@ -177,7 +177,7 @@ TEST logging_test(void) {
|
||||||
utarray_push_back(connections, &socket_fd);
|
utarray_push_back(connections, &socket_fd);
|
||||||
|
|
||||||
/* Start connection to Redis. */
|
/* Start connection to Redis. */
|
||||||
db_handle *conn =
|
DBHandle *conn =
|
||||||
db_connect("127.0.0.1", 6379, "test_process", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "test_process", "127.0.0.1", 0, NULL);
|
||||||
db_attach(conn, loop, false);
|
db_attach(conn, loop, false);
|
||||||
|
|
||||||
|
@ -185,8 +185,8 @@ TEST logging_test(void) {
|
||||||
int client_fd = connect_ipc_sock(socket_pathname);
|
int client_fd = connect_ipc_sock(socket_pathname);
|
||||||
ASSERT(client_fd >= 0);
|
ASSERT(client_fd >= 0);
|
||||||
utarray_push_back(connections, &client_fd);
|
utarray_push_back(connections, &client_fd);
|
||||||
ray_logger *logger = init_ray_logger("worker", RAY_INFO, 0, &client_fd);
|
RayLogger *logger = RayLogger_init("worker", RAY_INFO, 0, &client_fd);
|
||||||
ray_log(logger, RAY_INFO, "TEST", "Message");
|
RayLogger_log(logger, RAY_INFO, "TEST", "Message");
|
||||||
|
|
||||||
event_loop_add_file(loop, socket_fd, EVENT_LOOP_READ, logging_accept_callback,
|
event_loop_add_file(loop, socket_fd, EVENT_LOOP_READ, logging_accept_callback,
|
||||||
conn);
|
conn);
|
||||||
|
@ -197,7 +197,7 @@ TEST logging_test(void) {
|
||||||
|
|
||||||
ASSERT(logging_test_callback_called);
|
ASSERT(logging_test_callback_called);
|
||||||
|
|
||||||
free_ray_logger(logger);
|
RayLogger_free(logger);
|
||||||
db_disconnect(conn);
|
db_disconnect(conn);
|
||||||
event_loop_destroy(loop);
|
event_loop_destroy(loop);
|
||||||
for (int *p = (int *) utarray_front(connections); p != NULL;
|
for (int *p = (int *) utarray_front(connections); p != NULL;
|
||||||
|
|
|
@ -17,18 +17,18 @@ event_loop *g_loop;
|
||||||
|
|
||||||
/* === A lookup of a task not in the table === */
|
/* === A lookup of a task not in the table === */
|
||||||
|
|
||||||
task_id lookup_nil_id;
|
TaskID lookup_nil_id;
|
||||||
int lookup_nil_success = 0;
|
int lookup_nil_success = 0;
|
||||||
const char *lookup_nil_context = "lookup_nil";
|
const char *lookup_nil_context = "lookup_nil";
|
||||||
|
|
||||||
void lookup_nil_fail_callback(unique_id id,
|
void lookup_nil_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* The fail callback should not be called. */
|
/* The fail callback should not be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void lookup_nil_success_callback(task *task, void *context) {
|
void lookup_nil_success_callback(Task *task, void *context) {
|
||||||
lookup_nil_success = 1;
|
lookup_nil_success = 1;
|
||||||
CHECK(task == NULL);
|
CHECK(task == NULL);
|
||||||
CHECK(context == (void *) lookup_nil_context);
|
CHECK(context == (void *) lookup_nil_context);
|
||||||
|
@ -38,10 +38,10 @@ void lookup_nil_success_callback(task *task, void *context) {
|
||||||
TEST lookup_nil_test(void) {
|
TEST lookup_nil_test(void) {
|
||||||
lookup_nil_id = globally_unique_id();
|
lookup_nil_id = globally_unique_id();
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 1000,
|
.timeout = 1000,
|
||||||
.fail_callback = lookup_nil_fail_callback,
|
.fail_callback = lookup_nil_fail_callback,
|
||||||
|
@ -61,28 +61,28 @@ TEST lookup_nil_test(void) {
|
||||||
|
|
||||||
int add_success = 0;
|
int add_success = 0;
|
||||||
int lookup_success = 0;
|
int lookup_success = 0;
|
||||||
task *add_lookup_task;
|
Task *add_lookup_task;
|
||||||
const char *add_lookup_context = "add_lookup";
|
const char *add_lookup_context = "add_lookup";
|
||||||
|
|
||||||
void add_lookup_fail_callback(unique_id id,
|
void add_lookup_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* The fail callback should not be called. */
|
/* The fail callback should not be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void lookup_success_callback(task *task, void *context) {
|
void lookup_success_callback(Task *task, void *context) {
|
||||||
lookup_success = 1;
|
lookup_success = 1;
|
||||||
CHECK(memcmp(task, add_lookup_task, task_size(task)) == 0);
|
CHECK(memcmp(task, add_lookup_task, Task_size(task)) == 0);
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_success_callback(task_id task_id, void *context) {
|
void add_success_callback(TaskID task_id, void *context) {
|
||||||
add_success = 1;
|
add_success = 1;
|
||||||
CHECK(task_ids_equal(task_id, task_task_id(add_lookup_task)));
|
CHECK(TaskID_equal(task_id, Task_task_id(add_lookup_task)));
|
||||||
|
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 1000,
|
.timeout = 1000,
|
||||||
.fail_callback = add_lookup_fail_callback,
|
.fail_callback = add_lookup_fail_callback,
|
||||||
|
@ -94,15 +94,15 @@ void add_success_callback(task_id task_id, void *context) {
|
||||||
TEST add_lookup_test(void) {
|
TEST add_lookup_test(void) {
|
||||||
add_lookup_task = example_task(1, 1, TASK_STATUS_WAITING);
|
add_lookup_task = example_task(1, 1, TASK_STATUS_WAITING);
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 1000,
|
.timeout = 1000,
|
||||||
.fail_callback = add_lookup_fail_callback,
|
.fail_callback = add_lookup_fail_callback,
|
||||||
};
|
};
|
||||||
task_table_add_task(db, copy_task(add_lookup_task), &retry,
|
task_table_add_task(db, Task_copy(add_lookup_task), &retry,
|
||||||
add_success_callback, (void *) db);
|
add_success_callback, (void *) db);
|
||||||
/* Disconnect the database to see if the lookup times out. */
|
/* Disconnect the database to see if the lookup times out. */
|
||||||
event_loop_run(g_loop);
|
event_loop_run(g_loop);
|
||||||
|
@ -121,14 +121,12 @@ TEST add_lookup_test(void) {
|
||||||
const char *subscribe_timeout_context = "subscribe_timeout";
|
const char *subscribe_timeout_context = "subscribe_timeout";
|
||||||
int subscribe_failed = 0;
|
int subscribe_failed = 0;
|
||||||
|
|
||||||
void subscribe_done_callback(task_id task_id, void *user_context) {
|
void subscribe_done_callback(TaskID task_id, void *user_context) {
|
||||||
/* The done callback should not be called. */
|
/* The done callback should not be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_fail_callback(unique_id id,
|
void subscribe_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
void *user_context,
|
|
||||||
void *user_data) {
|
|
||||||
subscribe_failed = 1;
|
subscribe_failed = 1;
|
||||||
CHECK(user_context == (void *) subscribe_timeout_context);
|
CHECK(user_context == (void *) subscribe_timeout_context);
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
|
@ -136,10 +134,10 @@ void subscribe_fail_callback(unique_id id,
|
||||||
|
|
||||||
TEST subscribe_timeout_test(void) {
|
TEST subscribe_timeout_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = subscribe_fail_callback,
|
.fail_callback = subscribe_fail_callback,
|
||||||
|
@ -164,12 +162,12 @@ const char *publish_timeout_context = "publish_timeout";
|
||||||
const int publish_test_number = 272;
|
const int publish_test_number = 272;
|
||||||
int publish_failed = 0;
|
int publish_failed = 0;
|
||||||
|
|
||||||
void publish_done_callback(task_id task_id, void *user_context) {
|
void publish_done_callback(TaskID task_id, void *user_context) {
|
||||||
/* The done callback should not be called. */
|
/* The done callback should not be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void publish_fail_callback(unique_id id, void *user_context, void *user_data) {
|
void publish_fail_callback(UniqueID id, void *user_context, void *user_data) {
|
||||||
publish_failed = 1;
|
publish_failed = 1;
|
||||||
CHECK(user_context == (void *) publish_timeout_context);
|
CHECK(user_context == (void *) publish_timeout_context);
|
||||||
event_loop_stop(g_loop);
|
event_loop_stop(g_loop);
|
||||||
|
@ -177,11 +175,11 @@ void publish_fail_callback(unique_id id, void *user_context, void *user_data) {
|
||||||
|
|
||||||
TEST publish_timeout_test(void) {
|
TEST publish_timeout_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
Task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5, .timeout = 100, .fail_callback = publish_fail_callback,
|
.num_retries = 5, .timeout = 100, .fail_callback = publish_fail_callback,
|
||||||
};
|
};
|
||||||
task_table_add_task(db, task, &retry, publish_done_callback,
|
task_table_add_task(db, task, &retry, publish_done_callback,
|
||||||
|
@ -202,7 +200,7 @@ TEST publish_timeout_test(void) {
|
||||||
int64_t reconnect_db_callback(event_loop *loop,
|
int64_t reconnect_db_callback(event_loop *loop,
|
||||||
int64_t timer_id,
|
int64_t timer_id,
|
||||||
void *context) {
|
void *context) {
|
||||||
db_handle *db = context;
|
DBHandle *db = context;
|
||||||
/* Reconnect to redis. */
|
/* Reconnect to redis. */
|
||||||
redisAsyncFree(db->sub_context);
|
redisAsyncFree(db->sub_context);
|
||||||
db->sub_context = redisAsyncConnect("127.0.0.1", 6379);
|
db->sub_context = redisAsyncConnect("127.0.0.1", 6379);
|
||||||
|
@ -225,12 +223,12 @@ const char *subscribe_retry_context = "subscribe_retry";
|
||||||
const int subscribe_retry_test_number = 273;
|
const int subscribe_retry_test_number = 273;
|
||||||
int subscribe_retry_succeeded = 0;
|
int subscribe_retry_succeeded = 0;
|
||||||
|
|
||||||
void subscribe_retry_done_callback(object_id object_id, void *user_context) {
|
void subscribe_retry_done_callback(ObjectID object_id, void *user_context) {
|
||||||
CHECK(user_context == (void *) subscribe_retry_context);
|
CHECK(user_context == (void *) subscribe_retry_context);
|
||||||
subscribe_retry_succeeded = 1;
|
subscribe_retry_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_retry_fail_callback(unique_id id,
|
void subscribe_retry_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* The fail callback should not be called. */
|
/* The fail callback should not be called. */
|
||||||
|
@ -239,10 +237,10 @@ void subscribe_retry_fail_callback(unique_id id,
|
||||||
|
|
||||||
TEST subscribe_retry_test(void) {
|
TEST subscribe_retry_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = subscribe_retry_fail_callback,
|
.fail_callback = subscribe_retry_fail_callback,
|
||||||
|
@ -272,12 +270,12 @@ TEST subscribe_retry_test(void) {
|
||||||
const char *publish_retry_context = "publish_retry";
|
const char *publish_retry_context = "publish_retry";
|
||||||
int publish_retry_succeeded = 0;
|
int publish_retry_succeeded = 0;
|
||||||
|
|
||||||
void publish_retry_done_callback(object_id object_id, void *user_context) {
|
void publish_retry_done_callback(ObjectID object_id, void *user_context) {
|
||||||
CHECK(user_context == (void *) publish_retry_context);
|
CHECK(user_context == (void *) publish_retry_context);
|
||||||
publish_retry_succeeded = 1;
|
publish_retry_succeeded = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void publish_retry_fail_callback(unique_id id,
|
void publish_retry_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* The fail callback should not be called. */
|
/* The fail callback should not be called. */
|
||||||
|
@ -286,11 +284,11 @@ void publish_retry_fail_callback(unique_id id,
|
||||||
|
|
||||||
TEST publish_retry_test(void) {
|
TEST publish_retry_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
Task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 5,
|
.num_retries = 5,
|
||||||
.timeout = 100,
|
.timeout = 100,
|
||||||
.fail_callback = publish_retry_fail_callback,
|
.fail_callback = publish_retry_fail_callback,
|
||||||
|
@ -321,24 +319,24 @@ TEST publish_retry_test(void) {
|
||||||
const char *subscribe_late_context = "subscribe_late";
|
const char *subscribe_late_context = "subscribe_late";
|
||||||
int subscribe_late_failed = 0;
|
int subscribe_late_failed = 0;
|
||||||
|
|
||||||
void subscribe_late_fail_callback(unique_id id,
|
void subscribe_late_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
CHECK(user_context == (void *) subscribe_late_context);
|
CHECK(user_context == (void *) subscribe_late_context);
|
||||||
subscribe_late_failed = 1;
|
subscribe_late_failed = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void subscribe_late_done_callback(task_id task_id, void *user_context) {
|
void subscribe_late_done_callback(TaskID task_id, void *user_context) {
|
||||||
/* This function should never be called. */
|
/* This function should never be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST subscribe_late_test(void) {
|
TEST subscribe_late_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0,
|
.num_retries = 0,
|
||||||
.timeout = 0,
|
.timeout = 0,
|
||||||
.fail_callback = subscribe_late_fail_callback,
|
.fail_callback = subscribe_late_fail_callback,
|
||||||
|
@ -366,25 +364,25 @@ TEST subscribe_late_test(void) {
|
||||||
const char *publish_late_context = "publish_late";
|
const char *publish_late_context = "publish_late";
|
||||||
int publish_late_failed = 0;
|
int publish_late_failed = 0;
|
||||||
|
|
||||||
void publish_late_fail_callback(unique_id id,
|
void publish_late_fail_callback(UniqueID id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
CHECK(user_context == (void *) publish_late_context);
|
CHECK(user_context == (void *) publish_late_context);
|
||||||
publish_late_failed = 1;
|
publish_late_failed = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
void publish_late_done_callback(task_id task_id, void *user_context) {
|
void publish_late_done_callback(TaskID task_id, void *user_context) {
|
||||||
/* This function should never be called. */
|
/* This function should never be called. */
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST publish_late_test(void) {
|
TEST publish_late_test(void) {
|
||||||
g_loop = event_loop_create();
|
g_loop = event_loop_create();
|
||||||
db_handle *db =
|
DBHandle *db =
|
||||||
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1", 0, NULL);
|
||||||
db_attach(db, g_loop, false);
|
db_attach(db, g_loop, false);
|
||||||
task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
Task *task = example_task(1, 1, TASK_STATUS_WAITING);
|
||||||
retry_info retry = {
|
RetryInfo retry = {
|
||||||
.num_retries = 0,
|
.num_retries = 0,
|
||||||
.timeout = 0,
|
.timeout = 0,
|
||||||
.fail_callback = publish_late_fail_callback,
|
.fail_callback = publish_late_fail_callback,
|
||||||
|
|
|
@ -12,17 +12,17 @@
|
||||||
SUITE(task_tests);
|
SUITE(task_tests);
|
||||||
|
|
||||||
TEST task_test(void) {
|
TEST task_test(void) {
|
||||||
task_id parent_task_id = globally_unique_id();
|
TaskID parent_task_id = globally_unique_id();
|
||||||
function_id func_id = globally_unique_id();
|
FunctionID func_id = globally_unique_id();
|
||||||
task_spec *spec = start_construct_task_spec(
|
task_spec *spec = start_construct_task_spec(
|
||||||
NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 4, 2, 10);
|
NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 4, 2, 10);
|
||||||
ASSERT(task_num_args(spec) == 4);
|
ASSERT(task_num_args(spec) == 4);
|
||||||
ASSERT(task_num_returns(spec) == 2);
|
ASSERT(task_num_returns(spec) == 2);
|
||||||
|
|
||||||
unique_id arg1 = globally_unique_id();
|
UniqueID arg1 = globally_unique_id();
|
||||||
ASSERT(task_args_add_ref(spec, arg1) == 0);
|
ASSERT(task_args_add_ref(spec, arg1) == 0);
|
||||||
ASSERT(task_args_add_val(spec, (uint8_t *) "hello", 5) == 1);
|
ASSERT(task_args_add_val(spec, (uint8_t *) "hello", 5) == 1);
|
||||||
unique_id arg2 = globally_unique_id();
|
UniqueID arg2 = globally_unique_id();
|
||||||
ASSERT(task_args_add_ref(spec, arg2) == 2);
|
ASSERT(task_args_add_ref(spec, arg2) == 2);
|
||||||
ASSERT(task_args_add_val(spec, (uint8_t *) "world", 5) == 3);
|
ASSERT(task_args_add_val(spec, (uint8_t *) "world", 5) == 3);
|
||||||
/* Finish constructing the spec. This constructs the task ID and the
|
/* Finish constructing the spec. This constructs the task ID and the
|
||||||
|
@ -32,11 +32,11 @@ TEST task_test(void) {
|
||||||
/* Check that the spec was constructed as expected. */
|
/* Check that the spec was constructed as expected. */
|
||||||
ASSERT(task_num_args(spec) == 4);
|
ASSERT(task_num_args(spec) == 4);
|
||||||
ASSERT(task_num_returns(spec) == 2);
|
ASSERT(task_num_returns(spec) == 2);
|
||||||
ASSERT(function_ids_equal(task_function(spec), func_id));
|
ASSERT(FunctionID_equal(task_function(spec), func_id));
|
||||||
ASSERT(object_ids_equal(task_arg_id(spec, 0), arg1));
|
ASSERT(ObjectID_equal(task_arg_id(spec, 0), arg1));
|
||||||
ASSERT(memcmp(task_arg_val(spec, 1), (uint8_t *) "hello",
|
ASSERT(memcmp(task_arg_val(spec, 1), (uint8_t *) "hello",
|
||||||
task_arg_length(spec, 1)) == 0);
|
task_arg_length(spec, 1)) == 0);
|
||||||
ASSERT(object_ids_equal(task_arg_id(spec, 2), arg2));
|
ASSERT(ObjectID_equal(task_arg_id(spec, 2), arg2));
|
||||||
ASSERT(memcmp(task_arg_val(spec, 3), (uint8_t *) "world",
|
ASSERT(memcmp(task_arg_val(spec, 3), (uint8_t *) "world",
|
||||||
task_arg_length(spec, 3)) == 0);
|
task_arg_length(spec, 3)) == 0);
|
||||||
|
|
||||||
|
@ -46,9 +46,9 @@ TEST task_test(void) {
|
||||||
|
|
||||||
TEST deterministic_ids_test(void) {
|
TEST deterministic_ids_test(void) {
|
||||||
/* Define the inputs to the task construction. */
|
/* Define the inputs to the task construction. */
|
||||||
task_id parent_task_id = globally_unique_id();
|
TaskID parent_task_id = globally_unique_id();
|
||||||
function_id func_id = globally_unique_id();
|
FunctionID func_id = globally_unique_id();
|
||||||
unique_id arg1 = globally_unique_id();
|
UniqueID arg1 = globally_unique_id();
|
||||||
uint8_t *arg2 = (uint8_t *) "hello world";
|
uint8_t *arg2 = (uint8_t *) "hello world";
|
||||||
|
|
||||||
/* Construct a first task. */
|
/* Construct a first task. */
|
||||||
|
@ -66,14 +66,14 @@ TEST deterministic_ids_test(void) {
|
||||||
finish_construct_task_spec(spec2);
|
finish_construct_task_spec(spec2);
|
||||||
|
|
||||||
/* Check that these tasks have the same task IDs and the same return IDs.*/
|
/* Check that these tasks have the same task IDs and the same return IDs.*/
|
||||||
ASSERT(task_ids_equal(task_spec_id(spec1), task_spec_id(spec2)));
|
ASSERT(TaskID_equal(task_spec_id(spec1), task_spec_id(spec2)));
|
||||||
ASSERT(object_ids_equal(task_return(spec1, 0), task_return(spec2, 0)));
|
ASSERT(ObjectID_equal(task_return(spec1, 0), task_return(spec2, 0)));
|
||||||
ASSERT(object_ids_equal(task_return(spec1, 1), task_return(spec2, 1)));
|
ASSERT(ObjectID_equal(task_return(spec1, 1), task_return(spec2, 1)));
|
||||||
ASSERT(object_ids_equal(task_return(spec1, 2), task_return(spec2, 2)));
|
ASSERT(ObjectID_equal(task_return(spec1, 2), task_return(spec2, 2)));
|
||||||
/* Check that the return IDs are all distinct. */
|
/* Check that the return IDs are all distinct. */
|
||||||
ASSERT(!object_ids_equal(task_return(spec1, 0), task_return(spec2, 1)));
|
ASSERT(!ObjectID_equal(task_return(spec1, 0), task_return(spec2, 1)));
|
||||||
ASSERT(!object_ids_equal(task_return(spec1, 0), task_return(spec2, 2)));
|
ASSERT(!ObjectID_equal(task_return(spec1, 0), task_return(spec2, 2)));
|
||||||
ASSERT(!object_ids_equal(task_return(spec1, 1), task_return(spec2, 2)));
|
ASSERT(!ObjectID_equal(task_return(spec1, 1), task_return(spec2, 2)));
|
||||||
|
|
||||||
/* Create more tasks that are only mildly different. */
|
/* Create more tasks that are only mildly different. */
|
||||||
|
|
||||||
|
@ -114,11 +114,11 @@ TEST deterministic_ids_test(void) {
|
||||||
finish_construct_task_spec(spec7);
|
finish_construct_task_spec(spec7);
|
||||||
|
|
||||||
/* Check that the task IDs are all distinct from the original. */
|
/* Check that the task IDs are all distinct from the original. */
|
||||||
ASSERT(!task_ids_equal(task_spec_id(spec1), task_spec_id(spec3)));
|
ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec3)));
|
||||||
ASSERT(!task_ids_equal(task_spec_id(spec1), task_spec_id(spec4)));
|
ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec4)));
|
||||||
ASSERT(!task_ids_equal(task_spec_id(spec1), task_spec_id(spec5)));
|
ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec5)));
|
||||||
ASSERT(!task_ids_equal(task_spec_id(spec1), task_spec_id(spec6)));
|
ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec6)));
|
||||||
ASSERT(!task_ids_equal(task_spec_id(spec1), task_spec_id(spec7)));
|
ASSERT(!TaskID_equal(task_spec_id(spec1), task_spec_id(spec7)));
|
||||||
|
|
||||||
/* Check that the return object IDs are distinct from the originals. */
|
/* Check that the return object IDs are distinct from the originals. */
|
||||||
task_spec *specs[6] = {spec1, spec3, spec4, spec5, spec6, spec7};
|
task_spec *specs[6] = {spec1, spec3, spec4, spec5, spec6, spec7};
|
||||||
|
@ -127,7 +127,7 @@ TEST deterministic_ids_test(void) {
|
||||||
for (int task_index2 = 0; task_index2 < 6; ++task_index2) {
|
for (int task_index2 = 0; task_index2 < 6; ++task_index2) {
|
||||||
for (int return_index2 = 0; return_index2 < 3; ++return_index2) {
|
for (int return_index2 = 0; return_index2 < 3; ++return_index2) {
|
||||||
if (task_index1 != task_index2 && return_index1 != return_index2) {
|
if (task_index1 != task_index2 && return_index1 != return_index2) {
|
||||||
ASSERT(!object_ids_equal(
|
ASSERT(!ObjectID_equal(
|
||||||
task_return(specs[task_index1], return_index1),
|
task_return(specs[task_index1], return_index1),
|
||||||
task_return(specs[task_index2], return_index2)));
|
task_return(specs[task_index2], return_index2)));
|
||||||
}
|
}
|
||||||
|
@ -147,8 +147,8 @@ TEST deterministic_ids_test(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST send_task(void) {
|
TEST send_task(void) {
|
||||||
task_id parent_task_id = globally_unique_id();
|
TaskID parent_task_id = globally_unique_id();
|
||||||
function_id func_id = globally_unique_id();
|
FunctionID func_id = globally_unique_id();
|
||||||
task_spec *spec = start_construct_task_spec(
|
task_spec *spec = start_construct_task_spec(
|
||||||
NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0, func_id, 4, 2, 10);
|
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_ref(spec, globally_unique_id());
|
||||||
|
|
|
@ -18,14 +18,14 @@ const int64_t arg_value_size = 1000;
|
||||||
|
|
||||||
static inline task_spec *example_task_spec_with_args(int64_t num_args,
|
static inline task_spec *example_task_spec_with_args(int64_t num_args,
|
||||||
int64_t num_returns,
|
int64_t num_returns,
|
||||||
object_id arg_ids[]) {
|
ObjectID arg_ids[]) {
|
||||||
task_id parent_task_id = globally_unique_id();
|
TaskID parent_task_id = globally_unique_id();
|
||||||
function_id func_id = globally_unique_id();
|
FunctionID func_id = globally_unique_id();
|
||||||
task_spec *task =
|
task_spec *task =
|
||||||
start_construct_task_spec(NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0,
|
start_construct_task_spec(NIL_ID, parent_task_id, 0, NIL_ACTOR_ID, 0,
|
||||||
func_id, num_args, num_returns, arg_value_size);
|
func_id, num_args, num_returns, arg_value_size);
|
||||||
for (int64_t i = 0; i < num_args; ++i) {
|
for (int64_t i = 0; i < num_args; ++i) {
|
||||||
object_id arg_id;
|
ObjectID arg_id;
|
||||||
if (arg_ids == NULL) {
|
if (arg_ids == NULL) {
|
||||||
arg_id = globally_unique_id();
|
arg_id = globally_unique_id();
|
||||||
} else {
|
} else {
|
||||||
|
@ -42,21 +42,21 @@ static inline task_spec *example_task_spec(int64_t num_args,
|
||||||
return example_task_spec_with_args(num_args, num_returns, NULL);
|
return example_task_spec_with_args(num_args, num_returns, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
static inline task *example_task_with_args(int64_t num_args,
|
static inline Task *example_task_with_args(int64_t num_args,
|
||||||
int64_t num_returns,
|
int64_t num_returns,
|
||||||
int task_state,
|
int Task_state,
|
||||||
object_id arg_ids[]) {
|
ObjectID arg_ids[]) {
|
||||||
task_spec *spec = example_task_spec_with_args(num_args, num_returns, arg_ids);
|
task_spec *spec = example_task_spec_with_args(num_args, num_returns, arg_ids);
|
||||||
task *instance = alloc_task(spec, task_state, NIL_ID);
|
Task *instance = Task_alloc(spec, Task_state, NIL_ID);
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
|
||||||
static inline task *example_task(int64_t num_args,
|
static inline Task *example_task(int64_t num_args,
|
||||||
int64_t num_returns,
|
int64_t num_returns,
|
||||||
int task_state) {
|
int Task_state) {
|
||||||
task_spec *spec = example_task_spec(num_args, num_returns);
|
task_spec *spec = example_task_spec(num_args, num_returns);
|
||||||
task *instance = alloc_task(spec, task_state, NIL_ID);
|
Task *instance = Task_alloc(spec, Task_state, NIL_ID);
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,11 +15,11 @@
|
||||||
#include "state/task_table.h"
|
#include "state/task_table.h"
|
||||||
|
|
||||||
/* This is used to define the array of local schedulers used to define the
|
/* This is used to define the array of local schedulers used to define the
|
||||||
* global_scheduler_state type. */
|
* GlobalSchedulerState type. */
|
||||||
UT_icd local_scheduler_icd = {sizeof(local_scheduler), NULL, NULL, NULL};
|
UT_icd local_scheduler_icd = {sizeof(LocalScheduler), NULL, NULL, NULL};
|
||||||
|
|
||||||
/* This is used to define the array of tasks that haven't been scheduled yet. */
|
/* This is used to define the array of tasks that haven't been scheduled yet. */
|
||||||
UT_icd pending_tasks_icd = {sizeof(task *), NULL, NULL, NULL};
|
UT_icd pending_tasks_icd = {sizeof(Task *), NULL, NULL, NULL};
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Assign the given task to the local scheduler, update Redis and scheduler data
|
* Assign the given task to the local scheduler, update Redis and scheduler data
|
||||||
|
@ -30,24 +30,24 @@ UT_icd pending_tasks_icd = {sizeof(task *), NULL, NULL, NULL};
|
||||||
* @param local_scheduler_id DB client ID for the local scheduler.
|
* @param local_scheduler_id DB client ID for the local scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void assign_task_to_local_scheduler(global_scheduler_state *state,
|
void assign_task_to_local_scheduler(GlobalSchedulerState *state,
|
||||||
task *task,
|
Task *task,
|
||||||
db_client_id local_scheduler_id) {
|
DBClientID local_scheduler_id) {
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
task_spec *spec = task_task_spec(task);
|
task_spec *spec = Task_task_spec(task);
|
||||||
LOG_DEBUG("assigning task to local_scheduler_id = %s",
|
LOG_DEBUG("assigning task to local_scheduler_id = %s",
|
||||||
object_id_to_string(local_scheduler_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(local_scheduler_id, id_string, ID_STRING_SIZE));
|
||||||
task_set_state(task, TASK_STATUS_SCHEDULED);
|
Task_set_state(task, TASK_STATUS_SCHEDULED);
|
||||||
task_set_local_scheduler(task, local_scheduler_id);
|
Task_set_local_scheduler_id(task, local_scheduler_id);
|
||||||
LOG_DEBUG("Issuing a task table update for task = %s",
|
LOG_DEBUG("Issuing a task table update for task = %s",
|
||||||
object_id_to_string(task_task_id(task), id_string, ID_STRING_SIZE));
|
ObjectID_to_string(Task_task_id(task), id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
task_table_update(state->db, copy_task(task), NULL, NULL, NULL);
|
task_table_update(state->db, Task_copy(task), NULL, NULL, NULL);
|
||||||
|
|
||||||
/* TODO(rkn): We should probably pass around local_scheduler struct pointers
|
/* TODO(rkn): We should probably pass around local_scheduler struct pointers
|
||||||
* instead of db_client_id objects. */
|
* instead of db_client_id objects. */
|
||||||
/* Update the local scheduler info. */
|
/* Update the local scheduler info. */
|
||||||
local_scheduler *local_scheduler =
|
LocalScheduler *local_scheduler =
|
||||||
get_local_scheduler(state, local_scheduler_id);
|
get_local_scheduler(state, local_scheduler_id);
|
||||||
local_scheduler->num_tasks_sent += 1;
|
local_scheduler->num_tasks_sent += 1;
|
||||||
local_scheduler->num_recent_tasks_sent += 1;
|
local_scheduler->num_recent_tasks_sent += 1;
|
||||||
|
@ -61,28 +61,28 @@ void assign_task_to_local_scheduler(global_scheduler_state *state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
global_scheduler_state *init_global_scheduler(event_loop *loop,
|
GlobalSchedulerState *GlobalSchedulerState_init(event_loop *loop,
|
||||||
const char *redis_addr,
|
const char *redis_addr,
|
||||||
int redis_port) {
|
int redis_port) {
|
||||||
global_scheduler_state *state = malloc(sizeof(global_scheduler_state));
|
GlobalSchedulerState *state = malloc(sizeof(GlobalSchedulerState));
|
||||||
/* Must initialize state to 0. Sets hashmap head(s) to NULL. */
|
/* Must initialize state to 0. Sets hashmap head(s) to NULL. */
|
||||||
memset(state, 0, sizeof(global_scheduler_state));
|
memset(state, 0, sizeof(GlobalSchedulerState));
|
||||||
state->db =
|
state->db =
|
||||||
db_connect(redis_addr, redis_port, "global_scheduler", ":", 0, NULL);
|
db_connect(redis_addr, redis_port, "global_scheduler", ":", 0, NULL);
|
||||||
db_attach(state->db, loop, false);
|
db_attach(state->db, loop, false);
|
||||||
utarray_new(state->local_schedulers, &local_scheduler_icd);
|
utarray_new(state->local_schedulers, &local_scheduler_icd);
|
||||||
state->policy_state = init_global_scheduler_policy();
|
state->policy_state = GlobalSchedulerPolicyState_init();
|
||||||
/* Initialize the array of tasks that have not been scheduled yet. */
|
/* Initialize the array of tasks that have not been scheduled yet. */
|
||||||
utarray_new(state->pending_tasks, &pending_tasks_icd);
|
utarray_new(state->pending_tasks, &pending_tasks_icd);
|
||||||
return state;
|
return state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_global_scheduler(global_scheduler_state *state) {
|
void GlobalSchedulerState_free(GlobalSchedulerState *state) {
|
||||||
aux_address_entry *entry, *tmp;
|
AuxAddressEntry *entry, *tmp;
|
||||||
|
|
||||||
db_disconnect(state->db);
|
db_disconnect(state->db);
|
||||||
utarray_free(state->local_schedulers);
|
utarray_free(state->local_schedulers);
|
||||||
destroy_global_scheduler_policy(state->policy_state);
|
GlobalSchedulerPolicyState_free(state->policy_state);
|
||||||
/* Delete the plasma to photon association map. */
|
/* Delete the plasma to photon association map. */
|
||||||
HASH_ITER(plasma_photon_hh, state->plasma_photon_map, entry, tmp) {
|
HASH_ITER(plasma_photon_hh, state->plasma_photon_map, entry, tmp) {
|
||||||
HASH_DELETE(plasma_photon_hh, state->plasma_photon_map, entry);
|
HASH_DELETE(plasma_photon_hh, state->plasma_photon_map, entry);
|
||||||
|
@ -99,7 +99,7 @@ void free_global_scheduler(global_scheduler_state *state) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Free the scheduler object info table. */
|
/* Free the scheduler object info table. */
|
||||||
scheduler_object_info *object_entry, *tmp_entry;
|
SchedulerObjectInfo *object_entry, *tmp_entry;
|
||||||
HASH_ITER(hh, state->scheduler_object_info_table, object_entry, tmp_entry) {
|
HASH_ITER(hh, state->scheduler_object_info_table, object_entry, tmp_entry) {
|
||||||
HASH_DELETE(hh, state->scheduler_object_info_table, object_entry);
|
HASH_DELETE(hh, state->scheduler_object_info_table, object_entry);
|
||||||
utarray_free(object_entry->object_locations);
|
utarray_free(object_entry->object_locations);
|
||||||
|
@ -113,8 +113,8 @@ void free_global_scheduler(global_scheduler_state *state) {
|
||||||
num_pending_tasks);
|
num_pending_tasks);
|
||||||
}
|
}
|
||||||
for (int i = 0; i < num_pending_tasks; ++i) {
|
for (int i = 0; i < num_pending_tasks; ++i) {
|
||||||
task **pending_task = (task **) utarray_eltptr(state->pending_tasks, i);
|
Task **pending_task = (Task **) utarray_eltptr(state->pending_tasks, i);
|
||||||
free_task(*pending_task);
|
Task_free(*pending_task);
|
||||||
}
|
}
|
||||||
utarray_free(state->pending_tasks);
|
utarray_free(state->pending_tasks);
|
||||||
/* Free the global scheduler state. */
|
/* Free the global scheduler state. */
|
||||||
|
@ -123,24 +123,24 @@ void free_global_scheduler(global_scheduler_state *state) {
|
||||||
|
|
||||||
/* We need this code so we can clean up when we get a SIGTERM signal. */
|
/* We need this code so we can clean up when we get a SIGTERM signal. */
|
||||||
|
|
||||||
global_scheduler_state *g_state;
|
GlobalSchedulerState *g_state;
|
||||||
|
|
||||||
void signal_handler(int signal) {
|
void signal_handler(int signal) {
|
||||||
if (signal == SIGTERM) {
|
if (signal == SIGTERM) {
|
||||||
free_global_scheduler(g_state);
|
GlobalSchedulerState_free(g_state);
|
||||||
exit(0);
|
exit(0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* End of the cleanup code. */
|
/* End of the cleanup code. */
|
||||||
|
|
||||||
local_scheduler *get_local_scheduler(global_scheduler_state *state,
|
LocalScheduler *get_local_scheduler(GlobalSchedulerState *state,
|
||||||
db_client_id photon_id) {
|
DBClientID photon_id) {
|
||||||
local_scheduler *local_scheduler_ptr;
|
LocalScheduler *local_scheduler_ptr;
|
||||||
for (int i = 0; i < utarray_len(state->local_schedulers); ++i) {
|
for (int i = 0; i < utarray_len(state->local_schedulers); ++i) {
|
||||||
local_scheduler_ptr =
|
local_scheduler_ptr =
|
||||||
(local_scheduler *) utarray_eltptr(state->local_schedulers, i);
|
(LocalScheduler *) utarray_eltptr(state->local_schedulers, i);
|
||||||
if (db_client_ids_equal(local_scheduler_ptr->id, photon_id)) {
|
if (DBClientID_equal(local_scheduler_ptr->id, photon_id)) {
|
||||||
LOG_DEBUG("photon_id matched cached local scheduler entry.");
|
LOG_DEBUG("photon_id matched cached local scheduler entry.");
|
||||||
return local_scheduler_ptr;
|
return local_scheduler_ptr;
|
||||||
}
|
}
|
||||||
|
@ -148,8 +148,8 @@ local_scheduler *get_local_scheduler(global_scheduler_state *state,
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_task_waiting(task *waiting_task, void *user_context) {
|
void process_task_waiting(Task *waiting_task, void *user_context) {
|
||||||
global_scheduler_state *state = (global_scheduler_state *) user_context;
|
GlobalSchedulerState *state = (GlobalSchedulerState *) user_context;
|
||||||
LOG_DEBUG("Task waiting callback is called.");
|
LOG_DEBUG("Task waiting callback is called.");
|
||||||
bool successfully_assigned =
|
bool successfully_assigned =
|
||||||
handle_task_waiting(state, state->policy_state, waiting_task);
|
handle_task_waiting(state, state->policy_state, waiting_task);
|
||||||
|
@ -157,7 +157,7 @@ void process_task_waiting(task *waiting_task, void *user_context) {
|
||||||
* task to the array of pending tasks. The global scheduler will periodically
|
* task to the array of pending tasks. The global scheduler will periodically
|
||||||
* resubmit the tasks in this array. */
|
* resubmit the tasks in this array. */
|
||||||
if (!successfully_assigned) {
|
if (!successfully_assigned) {
|
||||||
task *task_copy = copy_task(waiting_task);
|
Task *task_copy = Task_copy(waiting_task);
|
||||||
utarray_push_back(state->pending_tasks, &task_copy);
|
utarray_push_back(state->pending_tasks, &task_copy);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -167,20 +167,19 @@ void process_task_waiting(task *waiting_task, void *user_context) {
|
||||||
* @param aux_address: an ip:port pair for the plasma manager associated with
|
* @param aux_address: an ip:port pair for the plasma manager associated with
|
||||||
* this db client.
|
* this db client.
|
||||||
*/
|
*/
|
||||||
void process_new_db_client(db_client_id db_client_id,
|
void process_new_db_client(DBClientID db_client_id,
|
||||||
const char *client_type,
|
const char *client_type,
|
||||||
const char *aux_address,
|
const char *aux_address,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
global_scheduler_state *state = (global_scheduler_state *) user_context;
|
GlobalSchedulerState *state = (GlobalSchedulerState *) user_context;
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG("db client table callback for db client = %s",
|
LOG_DEBUG("db client table callback for db client = %s",
|
||||||
object_id_to_string(db_client_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(db_client_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
if (strncmp(client_type, "photon", strlen("photon")) == 0) {
|
if (strncmp(client_type, "photon", strlen("photon")) == 0) {
|
||||||
/* Add plasma_manager ip:port -> photon_db_client_id association to state.
|
/* Add plasma_manager ip:port -> photon_db_client_id association to state.
|
||||||
*/
|
*/
|
||||||
aux_address_entry *plasma_photon_entry =
|
AuxAddressEntry *plasma_photon_entry = calloc(1, sizeof(AuxAddressEntry));
|
||||||
calloc(1, sizeof(aux_address_entry));
|
|
||||||
plasma_photon_entry->aux_address = strdup(aux_address);
|
plasma_photon_entry->aux_address = strdup(aux_address);
|
||||||
plasma_photon_entry->photon_db_client_id = db_client_id;
|
plasma_photon_entry->photon_db_client_id = db_client_id;
|
||||||
HASH_ADD_KEYPTR(plasma_photon_hh, state->plasma_photon_map,
|
HASH_ADD_KEYPTR(plasma_photon_hh, state->plasma_photon_map,
|
||||||
|
@ -197,18 +196,18 @@ void process_new_db_client(db_client_id db_client_id,
|
||||||
#if (RAY_COMMON_LOG_LEVEL <= RAY_COMMON_DEBUG)
|
#if (RAY_COMMON_LOG_LEVEL <= RAY_COMMON_DEBUG)
|
||||||
{
|
{
|
||||||
/* Print the photon to plasma association map so far. */
|
/* Print the photon to plasma association map so far. */
|
||||||
aux_address_entry *entry, *tmp;
|
AuxAddressEntry *entry, *tmp;
|
||||||
LOG_DEBUG("Photon to Plasma hash map so far:");
|
LOG_DEBUG("Photon to Plasma hash map so far:");
|
||||||
HASH_ITER(plasma_photon_hh, state->plasma_photon_map, entry, tmp) {
|
HASH_ITER(plasma_photon_hh, state->plasma_photon_map, entry, tmp) {
|
||||||
LOG_DEBUG("%s -> %s", entry->aux_address,
|
LOG_DEBUG("%s -> %s", entry->aux_address,
|
||||||
object_id_to_string(entry->photon_db_client_id, id_string,
|
ObjectID_to_string(entry->photon_db_client_id, id_string,
|
||||||
ID_STRING_SIZE));
|
ID_STRING_SIZE));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
/* Add new local scheduler to the state. */
|
/* Add new local scheduler to the state. */
|
||||||
local_scheduler local_scheduler;
|
LocalScheduler local_scheduler;
|
||||||
local_scheduler.id = db_client_id;
|
local_scheduler.id = db_client_id;
|
||||||
local_scheduler.num_tasks_sent = 0;
|
local_scheduler.num_tasks_sent = 0;
|
||||||
local_scheduler.num_recent_tasks_sent = 0;
|
local_scheduler.num_recent_tasks_sent = 0;
|
||||||
|
@ -235,30 +234,31 @@ void process_new_db_client(db_client_id db_client_id,
|
||||||
* @param user_context The user context.
|
* @param user_context The user context.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_table_subscribe_callback(object_id object_id,
|
void object_table_subscribe_callback(ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
/* Extract global scheduler state from the callback context. */
|
/* Extract global scheduler state from the callback context. */
|
||||||
global_scheduler_state *state = (global_scheduler_state *) user_context;
|
GlobalSchedulerState *state = (GlobalSchedulerState *) user_context;
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG("object table subscribe callback for OBJECT = %s",
|
LOG_DEBUG("object table subscribe callback for OBJECT = %s",
|
||||||
object_id_to_string(object_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(object_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
LOG_DEBUG("\tManagers<%d>:", manager_count);
|
LOG_DEBUG("\tManagers<%d>:", manager_count);
|
||||||
for (int i = 0; i < manager_count; i++) {
|
for (int i = 0; i < manager_count; i++) {
|
||||||
LOG_DEBUG("\t\t%s", manager_vector[i]);
|
LOG_DEBUG("\t\t%s", manager_vector[i]);
|
||||||
}
|
}
|
||||||
scheduler_object_info *obj_info_entry = NULL;
|
SchedulerObjectInfo *obj_info_entry = NULL;
|
||||||
|
|
||||||
HASH_FIND(hh, state->scheduler_object_info_table, &object_id,
|
HASH_FIND(hh, state->scheduler_object_info_table, &object_id,
|
||||||
sizeof(object_id), obj_info_entry);
|
sizeof(object_id), obj_info_entry);
|
||||||
|
|
||||||
if (obj_info_entry == NULL) {
|
if (obj_info_entry == NULL) {
|
||||||
/* Construct a new object info hash table entry. */
|
/* Construct a new object info hash table entry. */
|
||||||
obj_info_entry = malloc(sizeof(scheduler_object_info));
|
obj_info_entry =
|
||||||
memset(obj_info_entry, 0, sizeof(scheduler_object_info));
|
(SchedulerObjectInfo *) malloc(sizeof(SchedulerObjectInfo));
|
||||||
|
memset(obj_info_entry, 0, sizeof(*obj_info_entry));
|
||||||
|
|
||||||
obj_info_entry->object_id = object_id;
|
obj_info_entry->object_id = object_id;
|
||||||
obj_info_entry->data_size = data_size;
|
obj_info_entry->data_size = data_size;
|
||||||
|
@ -266,7 +266,7 @@ void object_table_subscribe_callback(object_id object_id,
|
||||||
HASH_ADD(hh, state->scheduler_object_info_table, object_id,
|
HASH_ADD(hh, state->scheduler_object_info_table, object_id,
|
||||||
sizeof(obj_info_entry->object_id), obj_info_entry);
|
sizeof(obj_info_entry->object_id), obj_info_entry);
|
||||||
LOG_DEBUG("New object added to object_info_table with id = %s",
|
LOG_DEBUG("New object added to object_info_table with id = %s",
|
||||||
object_id_to_string(object_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(object_id, id_string, ID_STRING_SIZE));
|
||||||
LOG_DEBUG("\tmanager locations:");
|
LOG_DEBUG("\tmanager locations:");
|
||||||
for (int i = 0; i < manager_count; i++) {
|
for (int i = 0; i < manager_count; i++) {
|
||||||
LOG_DEBUG("\t\t%s", manager_vector[i]);
|
LOG_DEBUG("\t\t%s", manager_vector[i]);
|
||||||
|
@ -285,22 +285,22 @@ void object_table_subscribe_callback(object_id object_id,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void local_scheduler_table_handler(db_client_id client_id,
|
void local_scheduler_table_handler(DBClientID client_id,
|
||||||
local_scheduler_info info,
|
LocalSchedulerInfo info,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
/* Extract global scheduler state from the callback context. */
|
/* Extract global scheduler state from the callback context. */
|
||||||
global_scheduler_state *state = (global_scheduler_state *) user_context;
|
GlobalSchedulerState *state = (GlobalSchedulerState *) user_context;
|
||||||
UNUSED(state);
|
UNUSED(state);
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
"Local scheduler heartbeat from db_client_id %s",
|
"Local scheduler heartbeat from db_client_id %s",
|
||||||
object_id_to_string((object_id) client_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string((ObjectID) client_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
"total workers = %d, task queue length = %d, available workers = %d",
|
"total workers = %d, task queue length = %d, available workers = %d",
|
||||||
info.total_num_workers, info.task_queue_length, info.available_workers);
|
info.total_num_workers, info.task_queue_length, info.available_workers);
|
||||||
/* Update the local scheduler info struct. */
|
/* Update the local scheduler info struct. */
|
||||||
local_scheduler *local_scheduler_ptr = get_local_scheduler(state, client_id);
|
LocalScheduler *local_scheduler_ptr = get_local_scheduler(state, client_id);
|
||||||
if (local_scheduler_ptr != NULL) {
|
if (local_scheduler_ptr != NULL) {
|
||||||
/* Reset the number of tasks sent since the last heartbeat. */
|
/* Reset the number of tasks sent since the last heartbeat. */
|
||||||
local_scheduler_ptr->num_recent_tasks_sent = 0;
|
local_scheduler_ptr->num_recent_tasks_sent = 0;
|
||||||
|
@ -311,11 +311,11 @@ void local_scheduler_table_handler(db_client_id client_id,
|
||||||
}
|
}
|
||||||
|
|
||||||
int task_cleanup_handler(event_loop *loop, timer_id id, void *context) {
|
int task_cleanup_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
global_scheduler_state *state = context;
|
GlobalSchedulerState *state = context;
|
||||||
/* Loop over the pending tasks and resubmit them. */
|
/* Loop over the pending tasks and resubmit them. */
|
||||||
int64_t num_pending_tasks = utarray_len(state->pending_tasks);
|
int64_t num_pending_tasks = utarray_len(state->pending_tasks);
|
||||||
for (int64_t i = num_pending_tasks - 1; i >= 0; --i) {
|
for (int64_t i = num_pending_tasks - 1; i >= 0; --i) {
|
||||||
task **pending_task = (task **) utarray_eltptr(state->pending_tasks, i);
|
Task **pending_task = (Task **) utarray_eltptr(state->pending_tasks, i);
|
||||||
/* Pretend that the task has been resubmitted. */
|
/* Pretend that the task has been resubmitted. */
|
||||||
bool successfully_assigned =
|
bool successfully_assigned =
|
||||||
handle_task_waiting(state, state->policy_state, *pending_task);
|
handle_task_waiting(state, state->policy_state, *pending_task);
|
||||||
|
@ -332,7 +332,7 @@ int task_cleanup_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
|
|
||||||
void start_server(const char *redis_addr, int redis_port) {
|
void start_server(const char *redis_addr, int redis_port) {
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
g_state = init_global_scheduler(loop, redis_addr, redis_port);
|
g_state = GlobalSchedulerState_init(loop, redis_addr, redis_port);
|
||||||
/* TODO(rkn): subscribe to notifications from the object table. */
|
/* TODO(rkn): subscribe to notifications from the object table. */
|
||||||
/* Subscribe to notifications about new local schedulers. TODO(rkn): this
|
/* Subscribe to notifications about new local schedulers. TODO(rkn): this
|
||||||
* needs to also get all of the clients that registered with the database
|
* needs to also get all of the clients that registered with the database
|
||||||
|
|
|
@ -15,7 +15,7 @@
|
||||||
/** Contains all information that is associated with a local scheduler. */
|
/** Contains all information that is associated with a local scheduler. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the local scheduler in Redis. */
|
/** The ID of the local scheduler in Redis. */
|
||||||
db_client_id id;
|
DBClientID id;
|
||||||
/** The number of tasks sent from the global scheduler to this local
|
/** The number of tasks sent from the global scheduler to this local
|
||||||
* scheduler. */
|
* scheduler. */
|
||||||
int64_t num_tasks_sent;
|
int64_t num_tasks_sent;
|
||||||
|
@ -24,24 +24,24 @@ typedef struct {
|
||||||
int64_t num_recent_tasks_sent;
|
int64_t num_recent_tasks_sent;
|
||||||
/** The latest information about the local scheduler capacity. This is updated
|
/** The latest information about the local scheduler capacity. This is updated
|
||||||
* every time a new local scheduler heartbeat arrives. */
|
* every time a new local scheduler heartbeat arrives. */
|
||||||
local_scheduler_info info;
|
LocalSchedulerInfo info;
|
||||||
} local_scheduler;
|
} LocalScheduler;
|
||||||
|
|
||||||
typedef struct global_scheduler_policy_state global_scheduler_policy_state;
|
typedef struct GlobalSchedulerPolicyState GlobalSchedulerPolicyState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This defines a hash table used to cache information about different objects.
|
* This defines a hash table used to cache information about different objects.
|
||||||
*/
|
*/
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The object ID in question. */
|
/** The object ID in question. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** The size in bytes of the object. */
|
/** The size in bytes of the object. */
|
||||||
int64_t data_size;
|
int64_t data_size;
|
||||||
/** An array of object locations for this object. */
|
/** An array of object locations for this object. */
|
||||||
UT_array *object_locations;
|
UT_array *object_locations;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} scheduler_object_info;
|
} SchedulerObjectInfo;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A struct used for caching Photon to Plasma association.
|
* A struct used for caching Photon to Plasma association.
|
||||||
|
@ -50,12 +50,12 @@ typedef struct {
|
||||||
/** IP:port string for the plasma_manager. */
|
/** IP:port string for the plasma_manager. */
|
||||||
char *aux_address;
|
char *aux_address;
|
||||||
/** Photon db client id. */
|
/** Photon db client id. */
|
||||||
db_client_id photon_db_client_id;
|
DBClientID photon_db_client_id;
|
||||||
/** Plasma_manager ip:port -> photon_db_client_id. */
|
/** Plasma_manager ip:port -> photon_db_client_id. */
|
||||||
UT_hash_handle plasma_photon_hh;
|
UT_hash_handle plasma_photon_hh;
|
||||||
/** Photon_db_client_id -> plasma_manager ip:port. */
|
/** Photon_db_client_id -> plasma_manager ip:port. */
|
||||||
UT_hash_handle photon_plasma_hh;
|
UT_hash_handle photon_plasma_hh;
|
||||||
} aux_address_entry;
|
} AuxAddressEntry;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Global scheduler state structure.
|
* Global scheduler state structure.
|
||||||
|
@ -64,22 +64,22 @@ typedef struct {
|
||||||
/** The global scheduler event loop. */
|
/** The global scheduler event loop. */
|
||||||
event_loop *loop;
|
event_loop *loop;
|
||||||
/** The global state store database. */
|
/** The global state store database. */
|
||||||
db_handle *db;
|
DBHandle *db;
|
||||||
/** The local schedulers that are connected to Redis. TODO(rkn): This probably
|
/** The local schedulers that are connected to Redis. TODO(rkn): This probably
|
||||||
* needs to be a hashtable since we often look up the local_scheduler struct
|
* needs to be a hashtable since we often look up the local_scheduler struct
|
||||||
* based on its db_client_id. */
|
* based on its db_client_id. */
|
||||||
UT_array *local_schedulers;
|
UT_array *local_schedulers;
|
||||||
/** The state managed by the scheduling policy. */
|
/** The state managed by the scheduling policy. */
|
||||||
global_scheduler_policy_state *policy_state;
|
GlobalSchedulerPolicyState *policy_state;
|
||||||
/** The plasma_manager ip:port -> photon_db_client_id association. */
|
/** The plasma_manager ip:port -> photon_db_client_id association. */
|
||||||
aux_address_entry *plasma_photon_map;
|
AuxAddressEntry *plasma_photon_map;
|
||||||
/** The photon_db_client_id -> plasma_manager ip:port association. */
|
/** The photon_db_client_id -> plasma_manager ip:port association. */
|
||||||
aux_address_entry *photon_plasma_map;
|
AuxAddressEntry *photon_plasma_map;
|
||||||
/** Objects cached by this global scheduler instance. */
|
/** Objects cached by this global scheduler instance. */
|
||||||
scheduler_object_info *scheduler_object_info_table;
|
SchedulerObjectInfo *scheduler_object_info_table;
|
||||||
/** An array of tasks that haven't been scheduled yet. */
|
/** An array of tasks that haven't been scheduled yet. */
|
||||||
UT_array *pending_tasks;
|
UT_array *pending_tasks;
|
||||||
} global_scheduler_state;
|
} GlobalSchedulerState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a helper method to look up the local scheduler struct that
|
* This is a helper method to look up the local scheduler struct that
|
||||||
|
@ -90,8 +90,8 @@ typedef struct {
|
||||||
* @return The corresponding local scheduler struct. If the global scheduler is
|
* @return The corresponding local scheduler struct. If the global scheduler is
|
||||||
* not aware of the local scheduler, then this will be NULL.
|
* not aware of the local scheduler, then this will be NULL.
|
||||||
*/
|
*/
|
||||||
local_scheduler *get_local_scheduler(global_scheduler_state *state,
|
LocalScheduler *get_local_scheduler(GlobalSchedulerState *state,
|
||||||
db_client_id photon_id);
|
DBClientID photon_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Assign the given task to the local scheduler, update Redis and scheduler data
|
* Assign the given task to the local scheduler, update Redis and scheduler data
|
||||||
|
@ -102,8 +102,8 @@ local_scheduler *get_local_scheduler(global_scheduler_state *state,
|
||||||
* @param local_scheduler_id DB client ID for the local scheduler.
|
* @param local_scheduler_id DB client ID for the local scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void assign_task_to_local_scheduler(global_scheduler_state *state,
|
void assign_task_to_local_scheduler(GlobalSchedulerState *state,
|
||||||
task *task,
|
Task *task,
|
||||||
db_client_id local_scheduler_id);
|
DBClientID local_scheduler_id);
|
||||||
|
|
||||||
#endif /* GLOBAL_SCHEDULER_H */
|
#endif /* GLOBAL_SCHEDULER_H */
|
||||||
|
|
|
@ -6,9 +6,9 @@
|
||||||
|
|
||||||
#include "global_scheduler_algorithm.h"
|
#include "global_scheduler_algorithm.h"
|
||||||
|
|
||||||
global_scheduler_policy_state *init_global_scheduler_policy(void) {
|
GlobalSchedulerPolicyState *GlobalSchedulerPolicyState_init(void) {
|
||||||
global_scheduler_policy_state *policy_state =
|
GlobalSchedulerPolicyState *policy_state =
|
||||||
malloc(sizeof(global_scheduler_policy_state));
|
malloc(sizeof(GlobalSchedulerPolicyState));
|
||||||
policy_state->round_robin_index = 0;
|
policy_state->round_robin_index = 0;
|
||||||
|
|
||||||
int num_weight_elem =
|
int num_weight_elem =
|
||||||
|
@ -23,8 +23,7 @@ global_scheduler_policy_state *init_global_scheduler_policy(void) {
|
||||||
return policy_state;
|
return policy_state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_global_scheduler_policy(
|
void GlobalSchedulerPolicyState_free(GlobalSchedulerPolicyState *policy_state) {
|
||||||
global_scheduler_policy_state *policy_state) {
|
|
||||||
free(policy_state);
|
free(policy_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -36,7 +35,7 @@ void destroy_global_scheduler_policy(
|
||||||
* @return True if all tasks's resource constraints are satisfied. False
|
* @return True if all tasks's resource constraints are satisfied. False
|
||||||
* otherwise.
|
* otherwise.
|
||||||
*/
|
*/
|
||||||
bool constraints_satisfied_hard(const local_scheduler *scheduler,
|
bool constraints_satisfied_hard(const LocalScheduler *scheduler,
|
||||||
const task_spec *spec) {
|
const task_spec *spec) {
|
||||||
for (int i = 0; i < MAX_RESOURCE_INDEX; i++) {
|
for (int i = 0; i < MAX_RESOURCE_INDEX; i++) {
|
||||||
if (scheduler->info.static_resources[i] <
|
if (scheduler->info.static_resources[i] <
|
||||||
|
@ -51,13 +50,13 @@ bool constraints_satisfied_hard(const local_scheduler *scheduler,
|
||||||
* This is a helper method that assigns a task to the next local scheduler in a
|
* This is a helper method that assigns a task to the next local scheduler in a
|
||||||
* round robin fashion.
|
* round robin fashion.
|
||||||
*/
|
*/
|
||||||
void handle_task_round_robin(global_scheduler_state *state,
|
void handle_task_round_robin(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
task *task) {
|
Task *task) {
|
||||||
CHECKM(utarray_len(state->local_schedulers) > 0,
|
CHECKM(utarray_len(state->local_schedulers) > 0,
|
||||||
"No local schedulers. We currently don't handle this case.");
|
"No local schedulers. We currently don't handle this case.");
|
||||||
local_scheduler *scheduler = NULL;
|
LocalScheduler *scheduler = NULL;
|
||||||
task_spec *task_spec = task_task_spec(task);
|
task_spec *task_spec = Task_task_spec(task);
|
||||||
int i;
|
int i;
|
||||||
int num_retries = 1;
|
int num_retries = 1;
|
||||||
bool task_satisfied = false;
|
bool task_satisfied = false;
|
||||||
|
@ -67,7 +66,7 @@ void handle_task_round_robin(global_scheduler_state *state,
|
||||||
if (i == policy_state->round_robin_index) {
|
if (i == policy_state->round_robin_index) {
|
||||||
num_retries--;
|
num_retries--;
|
||||||
}
|
}
|
||||||
scheduler = (local_scheduler *) utarray_eltptr(state->local_schedulers, i);
|
scheduler = (LocalScheduler *) utarray_eltptr(state->local_schedulers, i);
|
||||||
task_satisfied = constraints_satisfied_hard(scheduler, task_spec);
|
task_satisfied = constraints_satisfied_hard(scheduler, task_spec);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -83,11 +82,11 @@ void handle_task_round_robin(global_scheduler_state *state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object_size_entry *create_object_size_hashmap(global_scheduler_state *state,
|
ObjectSizeEntry *create_object_size_hashmap(GlobalSchedulerState *state,
|
||||||
task_spec *task_spec,
|
task_spec *task_spec,
|
||||||
bool *has_args_by_ref,
|
bool *has_args_by_ref,
|
||||||
int64_t *task_data_size) {
|
int64_t *task_data_size) {
|
||||||
object_size_entry *s = NULL, *object_size_table = NULL;
|
ObjectSizeEntry *s = NULL, *object_size_table = NULL;
|
||||||
*task_data_size = 0;
|
*task_data_size = 0;
|
||||||
|
|
||||||
for (int i = 0; i < task_num_args(task_spec); i++) {
|
for (int i = 0; i < task_num_args(task_spec); i++) {
|
||||||
|
@ -99,9 +98,9 @@ object_size_entry *create_object_size_hashmap(global_scheduler_state *state,
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
*has_args_by_ref = true;
|
*has_args_by_ref = true;
|
||||||
object_id obj_id = task_arg_id(task_spec, i);
|
ObjectID obj_id = task_arg_id(task_spec, i);
|
||||||
/* Look up this object ID in the global scheduler object cache. */
|
/* Look up this object ID in the global scheduler object cache. */
|
||||||
scheduler_object_info *obj_info_entry = NULL;
|
SchedulerObjectInfo *obj_info_entry = NULL;
|
||||||
HASH_FIND(hh, state->scheduler_object_info_table, &obj_id, sizeof(obj_id),
|
HASH_FIND(hh, state->scheduler_object_info_table, &obj_id, sizeof(obj_id),
|
||||||
obj_info_entry);
|
obj_info_entry);
|
||||||
if (obj_info_entry == NULL) {
|
if (obj_info_entry == NULL) {
|
||||||
|
@ -119,7 +118,7 @@ object_size_entry *create_object_size_hashmap(global_scheduler_state *state,
|
||||||
char **p = NULL;
|
char **p = NULL;
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG("locations for an arg_by_ref obj_id = %s",
|
LOG_DEBUG("locations for an arg_by_ref obj_id = %s",
|
||||||
object_id_to_string(obj_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(obj_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
for (p = (char **) utarray_front(obj_info_entry->object_locations);
|
for (p = (char **) utarray_front(obj_info_entry->object_locations);
|
||||||
p != NULL;
|
p != NULL;
|
||||||
|
@ -132,7 +131,7 @@ object_size_entry *create_object_size_hashmap(global_scheduler_state *state,
|
||||||
HASH_FIND_STR(object_size_table, object_location, s);
|
HASH_FIND_STR(object_size_table, object_location, s);
|
||||||
if (NULL == s) {
|
if (NULL == s) {
|
||||||
/* This location not yet known, so add this object location. */
|
/* This location not yet known, so add this object location. */
|
||||||
s = calloc(1, sizeof(object_size_entry));
|
s = calloc(1, sizeof(ObjectSizeEntry));
|
||||||
s->object_location = object_location;
|
s->object_location = object_location;
|
||||||
HASH_ADD_KEYPTR(hh, object_size_table, s->object_location,
|
HASH_ADD_KEYPTR(hh, object_size_table, s->object_location,
|
||||||
strlen(s->object_location), s);
|
strlen(s->object_location), s);
|
||||||
|
@ -145,9 +144,9 @@ object_size_entry *create_object_size_hashmap(global_scheduler_state *state,
|
||||||
return object_size_table;
|
return object_size_table;
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_object_size_hashmap(object_size_entry *object_size_table) {
|
void free_object_size_hashmap(ObjectSizeEntry *object_size_table) {
|
||||||
/* Destroy local state. */
|
/* Destroy local state. */
|
||||||
object_size_entry *tmp, *s = NULL;
|
ObjectSizeEntry *tmp, *s = NULL;
|
||||||
HASH_ITER(hh, object_size_table, s, tmp) {
|
HASH_ITER(hh, object_size_table, s, tmp) {
|
||||||
HASH_DEL(object_size_table, s);
|
HASH_DEL(object_size_table, s);
|
||||||
/* NOTE: Do not free externally stored s->object_location. */
|
/* NOTE: Do not free externally stored s->object_location. */
|
||||||
|
@ -155,10 +154,10 @@ void free_object_size_hashmap(object_size_entry *object_size_table) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
db_client_id get_photon_id(global_scheduler_state *state,
|
DBClientID get_photon_id(GlobalSchedulerState *state,
|
||||||
const char *plasma_location) {
|
const char *plasma_location) {
|
||||||
aux_address_entry *aux_entry = NULL;
|
AuxAddressEntry *aux_entry = NULL;
|
||||||
db_client_id photon_id = NIL_ID;
|
DBClientID photon_id = NIL_ID;
|
||||||
if (plasma_location != NULL) {
|
if (plasma_location != NULL) {
|
||||||
LOG_DEBUG("max object size location found : %s", plasma_location);
|
LOG_DEBUG("max object size location found : %s", plasma_location);
|
||||||
/* Lookup association of plasma location to photon. */
|
/* Lookup association of plasma location to photon. */
|
||||||
|
@ -177,7 +176,7 @@ db_client_id get_photon_id(global_scheduler_state *state,
|
||||||
|
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG("photon ID found = %s",
|
LOG_DEBUG("photon ID found = %s",
|
||||||
object_id_to_string(photon_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(photon_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
|
|
||||||
if (IS_NIL_ID(photon_id)) {
|
if (IS_NIL_ID(photon_id)) {
|
||||||
|
@ -186,7 +185,7 @@ db_client_id get_photon_id(global_scheduler_state *state,
|
||||||
|
|
||||||
/* Check to make sure this photon_db_client_id matches one of the
|
/* Check to make sure this photon_db_client_id matches one of the
|
||||||
* schedulers. */
|
* schedulers. */
|
||||||
local_scheduler *local_scheduler_ptr = get_local_scheduler(state, photon_id);
|
LocalScheduler *local_scheduler_ptr = get_local_scheduler(state, photon_id);
|
||||||
if (local_scheduler_ptr == NULL) {
|
if (local_scheduler_ptr == NULL) {
|
||||||
LOG_WARN("photon_id didn't match any cached local scheduler entries");
|
LOG_WARN("photon_id didn't match any cached local scheduler entries");
|
||||||
}
|
}
|
||||||
|
@ -201,9 +200,9 @@ double inner_product(double a[], double b[], int size) {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
double calculate_object_size_fraction(global_scheduler_state *state,
|
double calculate_object_size_fraction(GlobalSchedulerState *state,
|
||||||
local_scheduler *scheduler,
|
LocalScheduler *scheduler,
|
||||||
object_size_entry *object_size_table,
|
ObjectSizeEntry *object_size_table,
|
||||||
int64_t total_task_object_size) {
|
int64_t total_task_object_size) {
|
||||||
/* Look up its cached object size in the hashmap, normalize by total object
|
/* Look up its cached object size in the hashmap, normalize by total object
|
||||||
* size for this task. */
|
* size for this task. */
|
||||||
|
@ -215,11 +214,11 @@ double calculate_object_size_fraction(global_scheduler_state *state,
|
||||||
* which is used as the key for object_size_table.
|
* which is used as the key for object_size_table.
|
||||||
* This uses the plasma aux address to locate the object_size this node
|
* This uses the plasma aux address to locate the object_size this node
|
||||||
* contributes. */
|
* contributes. */
|
||||||
aux_address_entry *photon_plasma_pair = NULL;
|
AuxAddressEntry *photon_plasma_pair = NULL;
|
||||||
HASH_FIND(photon_plasma_hh, state->photon_plasma_map, &(scheduler->id),
|
HASH_FIND(photon_plasma_hh, state->photon_plasma_map, &(scheduler->id),
|
||||||
sizeof(scheduler->id), photon_plasma_pair);
|
sizeof(scheduler->id), photon_plasma_pair);
|
||||||
if (photon_plasma_pair != NULL) {
|
if (photon_plasma_pair != NULL) {
|
||||||
object_size_entry *s = NULL;
|
ObjectSizeEntry *s = NULL;
|
||||||
/* Found this node's photon to plasma mapping. Use the corresponding
|
/* Found this node's photon to plasma mapping. Use the corresponding
|
||||||
* plasma key to see if this node has any cached objects for this task. */
|
* plasma key to see if this node has any cached objects for this task. */
|
||||||
HASH_FIND_STR(object_size_table, photon_plasma_pair->aux_address, s);
|
HASH_FIND_STR(object_size_table, photon_plasma_pair->aux_address, s);
|
||||||
|
@ -235,8 +234,8 @@ double calculate_object_size_fraction(global_scheduler_state *state,
|
||||||
return object_size_fraction;
|
return object_size_fraction;
|
||||||
}
|
}
|
||||||
|
|
||||||
double calculate_score_dynvec_normalized(global_scheduler_state *state,
|
double calculate_score_dynvec_normalized(GlobalSchedulerState *state,
|
||||||
local_scheduler *scheduler,
|
LocalScheduler *scheduler,
|
||||||
const task_spec *task_spec,
|
const task_spec *task_spec,
|
||||||
double object_size_fraction) {
|
double object_size_fraction) {
|
||||||
/* The object size fraction is now calculated for this (task,node) pair. */
|
/* The object size fraction is now calculated for this (task,node) pair. */
|
||||||
|
@ -261,22 +260,22 @@ double calculate_score_dynvec_normalized(global_scheduler_state *state,
|
||||||
return score;
|
return score;
|
||||||
}
|
}
|
||||||
|
|
||||||
double calculate_cost_pending(const global_scheduler_state *state,
|
double calculate_cost_pending(const GlobalSchedulerState *state,
|
||||||
const local_scheduler *scheduler) {
|
const LocalScheduler *scheduler) {
|
||||||
/* TODO: make sure that num_recent_tasks_sent is reset on each heartbeat. */
|
/* TODO: make sure that num_recent_tasks_sent is reset on each heartbeat. */
|
||||||
return scheduler->num_recent_tasks_sent + scheduler->info.task_queue_length;
|
return scheduler->num_recent_tasks_sent + scheduler->info.task_queue_length;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool handle_task_waiting(global_scheduler_state *state,
|
bool handle_task_waiting(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
task *task) {
|
Task *task) {
|
||||||
task_spec *task_spec = task_task_spec(task);
|
task_spec *task_spec = Task_task_spec(task);
|
||||||
|
|
||||||
CHECKM(task_spec != NULL,
|
CHECKM(task_spec != NULL,
|
||||||
"task wait handler encounted a task with NULL spec");
|
"task wait handler encounted a task with NULL spec");
|
||||||
/* Local hash table to keep track of aggregate object sizes per local
|
/* Local hash table to keep track of aggregate object sizes per local
|
||||||
* scheduler. */
|
* scheduler. */
|
||||||
object_size_entry *object_size_table = NULL;
|
ObjectSizeEntry *object_size_table = NULL;
|
||||||
bool has_args_by_ref = false;
|
bool has_args_by_ref = false;
|
||||||
bool task_feasible = false;
|
bool task_feasible = false;
|
||||||
/* The total size of the task's data. */
|
/* The total size of the task's data. */
|
||||||
|
@ -286,12 +285,12 @@ bool handle_task_waiting(global_scheduler_state *state,
|
||||||
state, task_spec, &has_args_by_ref, &task_object_size);
|
state, task_spec, &has_args_by_ref, &task_object_size);
|
||||||
|
|
||||||
/* Go through all the nodes, calculate the score for each, pick max score. */
|
/* Go through all the nodes, calculate the score for each, pick max score. */
|
||||||
local_scheduler *scheduler = NULL;
|
LocalScheduler *scheduler = NULL;
|
||||||
double best_photon_score = INT32_MIN;
|
double best_photon_score = INT32_MIN;
|
||||||
CHECKM(best_photon_score < 0, "We might have a floating point underflow");
|
CHECKM(best_photon_score < 0, "We might have a floating point underflow");
|
||||||
db_client_id best_photon_id = NIL_ID; /* best node to send this task */
|
DBClientID best_photon_id = NIL_ID; /* best node to send this task */
|
||||||
for (scheduler = (local_scheduler *) utarray_front(state->local_schedulers);
|
for (scheduler = (LocalScheduler *) utarray_front(state->local_schedulers);
|
||||||
scheduler != NULL; scheduler = (local_scheduler *) utarray_next(
|
scheduler != NULL; scheduler = (LocalScheduler *) utarray_next(
|
||||||
state->local_schedulers, scheduler)) {
|
state->local_schedulers, scheduler)) {
|
||||||
/* For each local scheduler, calculate its score. Check hard constraints
|
/* For each local scheduler, calculate its score. Check hard constraints
|
||||||
* first. */
|
* first. */
|
||||||
|
@ -313,7 +312,7 @@ bool handle_task_waiting(global_scheduler_state *state,
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_ERROR(
|
LOG_ERROR(
|
||||||
"Infeasible task. No nodes satisfy hard constraints for task = %s",
|
"Infeasible task. No nodes satisfy hard constraints for task = %s",
|
||||||
object_id_to_string(task_task_id(task), id_string, ID_STRING_SIZE));
|
ObjectID_to_string(Task_task_id(task), id_string, ID_STRING_SIZE));
|
||||||
/* TODO(atumanov): propagate this error to the task's driver and/or
|
/* TODO(atumanov): propagate this error to the task's driver and/or
|
||||||
* cache the task in case new local schedulers satisfy it in the future. */
|
* cache the task in case new local schedulers satisfy it in the future. */
|
||||||
return false;
|
return false;
|
||||||
|
@ -325,14 +324,14 @@ bool handle_task_waiting(global_scheduler_state *state,
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_object_available(global_scheduler_state *state,
|
void handle_object_available(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
/* Do nothing for now. */
|
/* Do nothing for now. */
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_new_local_scheduler(global_scheduler_state *state,
|
void handle_new_local_scheduler(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
db_client_id db_client_id) {
|
DBClientID db_client_id) {
|
||||||
/* Do nothing for now. */
|
/* Do nothing for now. */
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,7 @@ typedef enum {
|
||||||
} global_scheduler_algorithm;
|
} global_scheduler_algorithm;
|
||||||
|
|
||||||
/** The state managed by the global scheduling policy. */
|
/** The state managed by the global scheduling policy. */
|
||||||
struct global_scheduler_policy_state {
|
struct GlobalSchedulerPolicyState {
|
||||||
/** The index of the next local scheduler to assign a task to. */
|
/** The index of the next local scheduler to assign a task to. */
|
||||||
int64_t round_robin_index;
|
int64_t round_robin_index;
|
||||||
double resource_attribute_weight[MAX_RESOURCE_INDEX + 1];
|
double resource_attribute_weight[MAX_RESOURCE_INDEX + 1];
|
||||||
|
@ -30,7 +30,7 @@ typedef struct {
|
||||||
const char *object_location;
|
const char *object_location;
|
||||||
int64_t total_object_size;
|
int64_t total_object_size;
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} object_size_entry;
|
} ObjectSizeEntry;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the state of the global scheduler policy. This state must be freed by
|
* Create the state of the global scheduler policy. This state must be freed by
|
||||||
|
@ -38,7 +38,7 @@ typedef struct {
|
||||||
*
|
*
|
||||||
* @return The state of the scheduling policy.
|
* @return The state of the scheduling policy.
|
||||||
*/
|
*/
|
||||||
global_scheduler_policy_state *init_global_scheduler_policy(void);
|
GlobalSchedulerPolicyState *GlobalSchedulerPolicyState_init(void);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Free the global scheduler policy state.
|
* Free the global scheduler policy state.
|
||||||
|
@ -46,8 +46,7 @@ global_scheduler_policy_state *init_global_scheduler_policy(void);
|
||||||
* @param policy_state The policy state to free.
|
* @param policy_state The policy state to free.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void destroy_global_scheduler_policy(
|
void GlobalSchedulerPolicyState_free(GlobalSchedulerPolicyState *policy_state);
|
||||||
global_scheduler_policy_state *policy_state);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Main new task handling function in the global scheduler.
|
* Main new task handling function in the global scheduler.
|
||||||
|
@ -58,9 +57,9 @@ void destroy_global_scheduler_policy(
|
||||||
* @return True if the task was assigned to a local scheduler and false
|
* @return True if the task was assigned to a local scheduler and false
|
||||||
* otherwise.
|
* otherwise.
|
||||||
*/
|
*/
|
||||||
bool handle_task_waiting(global_scheduler_state *state,
|
bool handle_task_waiting(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
task *task);
|
Task *task);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle the fact that a new object is available.
|
* Handle the fact that a new object is available.
|
||||||
|
@ -70,9 +69,9 @@ bool handle_task_waiting(global_scheduler_state *state,
|
||||||
* @param object_id The ID of the object that is now available.
|
* @param object_id The ID of the object that is now available.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_object_available(global_scheduler_state *state,
|
void handle_object_available(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle a heartbeat message from a local scheduler. TODO(rkn): this is a
|
* Handle a heartbeat message from a local scheduler. TODO(rkn): this is a
|
||||||
|
@ -82,9 +81,8 @@ void handle_object_available(global_scheduler_state *state,
|
||||||
* @param policy_state The state managed by the scheduling policy.
|
* @param policy_state The state managed by the scheduling policy.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_local_scheduler_heartbeat(
|
void handle_local_scheduler_heartbeat(GlobalSchedulerState *state,
|
||||||
global_scheduler_state *state,
|
GlobalSchedulerPolicyState *policy_state);
|
||||||
global_scheduler_policy_state *policy_state);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle the presence of a new local scheduler. Currently, this just adds the
|
* Handle the presence of a new local scheduler. Currently, this just adds the
|
||||||
|
@ -95,8 +93,8 @@ void handle_local_scheduler_heartbeat(
|
||||||
* @param The db client ID of the new local scheduler.
|
* @param The db client ID of the new local scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_new_local_scheduler(global_scheduler_state *state,
|
void handle_new_local_scheduler(GlobalSchedulerState *state,
|
||||||
global_scheduler_policy_state *policy_state,
|
GlobalSchedulerPolicyState *policy_state,
|
||||||
db_client_id db_client_id);
|
DBClientID db_client_id);
|
||||||
|
|
||||||
#endif /* GLOBAL_SCHEDULER_ALGORITHM_H */
|
#endif /* GLOBAL_SCHEDULER_ALGORITHM_H */
|
||||||
|
|
|
@ -208,14 +208,14 @@ static PyObject* register_callbacks(PyObject* self, PyObject* args) {
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
static void BufferCapsule_Destructor(PyObject* capsule) {
|
static void BufferCapsule_Destructor(PyObject* capsule) {
|
||||||
object_id* id = reinterpret_cast<object_id*>(PyCapsule_GetPointer(capsule, "buffer"));
|
ObjectID* id = reinterpret_cast<ObjectID*>(PyCapsule_GetPointer(capsule, "buffer"));
|
||||||
auto context = reinterpret_cast<PyObject*>(PyCapsule_GetContext(capsule));
|
auto context = reinterpret_cast<PyObject*>(PyCapsule_GetContext(capsule));
|
||||||
/* We use the context of the connection capsule to indicate if the connection
|
/* We use the context of the connection capsule to indicate if the connection
|
||||||
* is still active (if the context is NULL) or if it is closed (if the context
|
* is still active (if the context is NULL) or if it is closed (if the context
|
||||||
* is (void*) 0x1). This is neccessary because the primary pointer of the
|
* is (void*) 0x1). This is neccessary because the primary pointer of the
|
||||||
* capsule cannot be NULL. */
|
* capsule cannot be NULL. */
|
||||||
if (PyCapsule_GetContext(context) == NULL) {
|
if (PyCapsule_GetContext(context) == NULL) {
|
||||||
plasma_connection* conn;
|
PlasmaConnection* conn;
|
||||||
CHECK(PyObjectToPlasmaConnection(context, &conn));
|
CHECK(PyObjectToPlasmaConnection(context, &conn));
|
||||||
plasma_release(conn, *id);
|
plasma_release(conn, *id);
|
||||||
}
|
}
|
||||||
|
@ -236,8 +236,8 @@ static void BufferCapsule_Destructor(PyObject* capsule) {
|
||||||
* @return None.
|
* @return None.
|
||||||
*/
|
*/
|
||||||
static PyObject* store_list(PyObject* self, PyObject* args) {
|
static PyObject* store_list(PyObject* self, PyObject* args) {
|
||||||
object_id obj_id;
|
ObjectID obj_id;
|
||||||
plasma_connection* conn;
|
PlasmaConnection* conn;
|
||||||
PyObject* value;
|
PyObject* value;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&O", PyStringToUniqueID, &obj_id,
|
if (!PyArg_ParseTuple(args, "O&O&O", PyStringToUniqueID, &obj_id,
|
||||||
PyObjectToPlasmaConnection, &conn, &value)) {
|
PyObjectToPlasmaConnection, &conn, &value)) {
|
||||||
|
@ -316,12 +316,12 @@ static PyObject* retrieve_list(PyObject* self, PyObject* args) {
|
||||||
if (!PyArg_ParseTuple(args, "OOL", &object_id_list, &plasma_conn, &timeout_ms)) {
|
if (!PyArg_ParseTuple(args, "OOL", &object_id_list, &plasma_conn, &timeout_ms)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
plasma_connection* conn;
|
PlasmaConnection* conn;
|
||||||
if (!PyObjectToPlasmaConnection(plasma_conn, &conn)) { return NULL; }
|
if (!PyObjectToPlasmaConnection(plasma_conn, &conn)) { return NULL; }
|
||||||
|
|
||||||
Py_ssize_t num_object_ids = PyList_Size(object_id_list);
|
Py_ssize_t num_object_ids = PyList_Size(object_id_list);
|
||||||
object_id* object_ids = new object_id[num_object_ids];
|
ObjectID* object_ids = new ObjectID[num_object_ids];
|
||||||
object_buffer* object_buffers = new object_buffer[num_object_ids];
|
ObjectBuffer* object_buffers = new ObjectBuffer[num_object_ids];
|
||||||
|
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
for (int i = 0; i < num_object_ids; ++i) {
|
||||||
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
||||||
|
@ -340,7 +340,7 @@ static PyObject* retrieve_list(PyObject* self, PyObject* args) {
|
||||||
|
|
||||||
if (object_buffers[i].data_size != -1) {
|
if (object_buffers[i].data_size != -1) {
|
||||||
/* The object was retrieved, so return the object. */
|
/* The object was retrieved, so return the object. */
|
||||||
object_id* buffer_obj_id = new object_id(object_ids[i]);
|
ObjectID* buffer_obj_id = new ObjectID(object_ids[i]);
|
||||||
/* This keeps a Plasma buffer in scope as long as an object that is backed by that
|
/* This keeps a Plasma buffer in scope as long as an object that is backed by that
|
||||||
* buffer is in scope. This prevents memory in the object store from getting
|
* buffer is in scope. This prevents memory in the object store from getting
|
||||||
* released while it is still being used to back a Python object. */
|
* released while it is still being used to back a Python object. */
|
||||||
|
|
|
@ -38,7 +38,7 @@ UT_icd pid_t_icd;
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the actor. This is NIL_ACTOR_ID if the worker is not an actor.
|
/** The ID of the actor. This is NIL_ACTOR_ID if the worker is not an actor.
|
||||||
*/
|
*/
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** The process ID of this worker. */
|
/** The process ID of this worker. */
|
||||||
pid_t worker_pid;
|
pid_t worker_pid;
|
||||||
} register_worker_info;
|
} register_worker_info;
|
||||||
|
@ -47,15 +47,15 @@ typedef struct {
|
||||||
* local scheduler that is responsible for the actor. */
|
* local scheduler that is responsible for the actor. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the actor. This is used as a key in the hash table. */
|
/** The ID of the actor. This is used as a key in the hash table. */
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** The ID of the local scheduler that is responsible for the actor. */
|
/** The ID of the local scheduler that is responsible for the actor. */
|
||||||
db_client_id local_scheduler_id;
|
DBClientID local_scheduler_id;
|
||||||
/** Handle fo the hash table. */
|
/** Handle fo the hash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} actor_map_entry;
|
} actor_map_entry;
|
||||||
|
|
||||||
/** Internal state of the scheduling algorithm. */
|
/** Internal state of the scheduling algorithm. */
|
||||||
typedef struct scheduling_algorithm_state scheduling_algorithm_state;
|
typedef struct SchedulingAlgorithmState SchedulingAlgorithmState;
|
||||||
|
|
||||||
/** A struct storing the configuration state of the local scheduler. This should
|
/** A struct storing the configuration state of the local scheduler. This should
|
||||||
* consist of values that don't change over the lifetime of the local
|
* consist of values that don't change over the lifetime of the local
|
||||||
|
@ -84,11 +84,11 @@ typedef struct {
|
||||||
* that is responsible for the actor. */
|
* that is responsible for the actor. */
|
||||||
actor_map_entry *actor_mapping;
|
actor_map_entry *actor_mapping;
|
||||||
/** The handle to the database. */
|
/** The handle to the database. */
|
||||||
db_handle *db;
|
DBHandle *db;
|
||||||
/** The Plasma client. */
|
/** The Plasma client. */
|
||||||
plasma_connection *plasma_conn;
|
PlasmaConnection *plasma_conn;
|
||||||
/** State for the scheduling algorithm. */
|
/** State for the scheduling algorithm. */
|
||||||
scheduling_algorithm_state *algorithm_state;
|
SchedulingAlgorithmState *algorithm_state;
|
||||||
/** Input buffer, used for reading input in process_message to avoid
|
/** Input buffer, used for reading input in process_message to avoid
|
||||||
* allocation for each call to process_message. */
|
* allocation for each call to process_message. */
|
||||||
UT_array *input_buffer;
|
UT_array *input_buffer;
|
||||||
|
@ -98,7 +98,7 @@ typedef struct {
|
||||||
/** Vector of dynamic attributes associated with the node owned by this local
|
/** Vector of dynamic attributes associated with the node owned by this local
|
||||||
* scheduler. */
|
* scheduler. */
|
||||||
double dynamic_resources[MAX_RESOURCE_INDEX];
|
double dynamic_resources[MAX_RESOURCE_INDEX];
|
||||||
} local_scheduler_state;
|
} LocalSchedulerState;
|
||||||
|
|
||||||
/** Contains all information associated with a local scheduler client. */
|
/** Contains all information associated with a local scheduler client. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
|
@ -107,7 +107,7 @@ typedef struct {
|
||||||
/** A pointer to the task object that is currently running on this client. If
|
/** A pointer to the task object that is currently running on this client. If
|
||||||
* no task is running on the worker, this will be NULL. This is used to
|
* no task is running on the worker, this will be NULL. This is used to
|
||||||
* update the task table. */
|
* update the task table. */
|
||||||
task *task_in_progress;
|
Task *task_in_progress;
|
||||||
/** A flag to indicate whether this worker is currently blocking on an
|
/** A flag to indicate whether this worker is currently blocking on an
|
||||||
* object(s) that isn't available locally yet. */
|
* object(s) that isn't available locally yet. */
|
||||||
bool is_blocked;
|
bool is_blocked;
|
||||||
|
@ -118,9 +118,9 @@ typedef struct {
|
||||||
bool is_child;
|
bool is_child;
|
||||||
/** The ID of the actor on this worker. If there is no actor running on this
|
/** The ID of the actor on this worker. If there is no actor running on this
|
||||||
* worker, this should be NIL_ACTOR_ID. */
|
* worker, this should be NIL_ACTOR_ID. */
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** A pointer to the local scheduler state. */
|
/** A pointer to the local scheduler state. */
|
||||||
local_scheduler_state *local_scheduler_state;
|
LocalSchedulerState *local_scheduler_state;
|
||||||
} local_scheduler_client;
|
} LocalSchedulerClient;
|
||||||
|
|
||||||
#endif /* PHOTON_H */
|
#endif /* PHOTON_H */
|
||||||
|
|
|
@ -12,8 +12,7 @@
|
||||||
#include "common/task.h"
|
#include "common/task.h"
|
||||||
|
|
||||||
/* Declared for convenience. */
|
/* Declared for convenience. */
|
||||||
void remove_actor(scheduling_algorithm_state *algorithm_state,
|
void remove_actor(SchedulingAlgorithmState *algorithm_state, ActorID actor_id);
|
||||||
actor_id actor_id);
|
|
||||||
|
|
||||||
typedef struct task_queue_entry {
|
typedef struct task_queue_entry {
|
||||||
/** The task that is queued. */
|
/** The task that is queued. */
|
||||||
|
@ -26,7 +25,7 @@ typedef struct task_queue_entry {
|
||||||
* which objects are being actively fetched. */
|
* which objects are being actively fetched. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Object id of this object. */
|
/** Object id of this object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** An array of the tasks dependent on this object. */
|
/** An array of the tasks dependent on this object. */
|
||||||
UT_array *dependent_tasks;
|
UT_array *dependent_tasks;
|
||||||
/** Handle for the uthash table. NOTE: This handle is used for both the
|
/** Handle for the uthash table. NOTE: This handle is used for both the
|
||||||
|
@ -42,13 +41,13 @@ UT_icd task_queue_entry_icd = {sizeof(task_queue_entry *), NULL, NULL, NULL};
|
||||||
* corresponding local scheduler is unknown. */
|
* corresponding local scheduler is unknown. */
|
||||||
UT_icd task_spec_icd = {sizeof(task_spec *), NULL, NULL, NULL};
|
UT_icd task_spec_icd = {sizeof(task_spec *), NULL, NULL, NULL};
|
||||||
/** This is used to define the queue of available workers. */
|
/** This is used to define the queue of available workers. */
|
||||||
UT_icd worker_icd = {sizeof(local_scheduler_client *), NULL, NULL, NULL};
|
UT_icd worker_icd = {sizeof(LocalSchedulerClient *), NULL, NULL, NULL};
|
||||||
|
|
||||||
/** This struct contains information about a specific actor. This struct will be
|
/** This struct contains information about a specific actor. This struct will be
|
||||||
* used inside of a hash table. */
|
* used inside of a hash table. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the actor. This is used as a key in the hash table. */
|
/** The ID of the actor. This is used as a key in the hash table. */
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
/** The number of tasks that have been executed on this actor so far. This is
|
/** The number of tasks that have been executed on this actor so far. This is
|
||||||
* used to guarantee the in-order execution of tasks on actors (in the order
|
* used to guarantee the in-order execution of tasks on actors (in the order
|
||||||
* that the tasks were submitted). This is currently meaningful because we
|
* that the tasks were submitted). This is currently meaningful because we
|
||||||
|
@ -59,15 +58,15 @@ typedef struct {
|
||||||
* the order of their actor counters. */
|
* the order of their actor counters. */
|
||||||
task_queue_entry *task_queue;
|
task_queue_entry *task_queue;
|
||||||
/** The worker that the actor is running on. */
|
/** The worker that the actor is running on. */
|
||||||
local_scheduler_client *worker;
|
LocalSchedulerClient *worker;
|
||||||
/** True if the worker is available and false otherwise. */
|
/** True if the worker is available and false otherwise. */
|
||||||
bool worker_available;
|
bool worker_available;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} local_actor_info;
|
} LocalActorInfo;
|
||||||
|
|
||||||
/** Part of the photon state that is maintained by the scheduling algorithm. */
|
/** Part of the photon state that is maintained by the scheduling algorithm. */
|
||||||
struct scheduling_algorithm_state {
|
struct SchedulingAlgorithmState {
|
||||||
/** An array of pointers to tasks that are waiting for dependencies. */
|
/** An array of pointers to tasks that are waiting for dependencies. */
|
||||||
task_queue_entry *waiting_task_queue;
|
task_queue_entry *waiting_task_queue;
|
||||||
/** An array of pointers to tasks whose dependencies are ready but that are
|
/** An array of pointers to tasks whose dependencies are ready but that are
|
||||||
|
@ -76,7 +75,7 @@ struct scheduling_algorithm_state {
|
||||||
/** This is a hash table from actor ID to information about that actor. In
|
/** This is a hash table from actor ID to information about that actor. In
|
||||||
* particular, a queue of tasks that are waiting to execute on that actor.
|
* particular, a queue of tasks that are waiting to execute on that actor.
|
||||||
* This is only used for actors that exist locally. */
|
* This is only used for actors that exist locally. */
|
||||||
local_actor_info *local_actor_infos;
|
LocalActorInfo *local_actor_infos;
|
||||||
/** An array of actor tasks that have been submitted but this local scheduler
|
/** An array of actor tasks that have been submitted but this local scheduler
|
||||||
* doesn't know which local scheduler is responsible for them, so cannot
|
* doesn't know which local scheduler is responsible for them, so cannot
|
||||||
* assign them to the correct local scheduler yet. Whenever a notification
|
* assign them to the correct local scheduler yet. Whenever a notification
|
||||||
|
@ -107,9 +106,9 @@ struct scheduling_algorithm_state {
|
||||||
object_entry *remote_objects;
|
object_entry *remote_objects;
|
||||||
};
|
};
|
||||||
|
|
||||||
scheduling_algorithm_state *make_scheduling_algorithm_state(void) {
|
SchedulingAlgorithmState *SchedulingAlgorithmState_init(void) {
|
||||||
scheduling_algorithm_state *algorithm_state =
|
SchedulingAlgorithmState *algorithm_state =
|
||||||
malloc(sizeof(scheduling_algorithm_state));
|
malloc(sizeof(SchedulingAlgorithmState));
|
||||||
/* Initialize an empty hash map for the cache of local available objects. */
|
/* Initialize an empty hash map for the cache of local available objects. */
|
||||||
algorithm_state->local_objects = NULL;
|
algorithm_state->local_objects = NULL;
|
||||||
/* Initialize the hash table of objects being fetched. */
|
/* Initialize the hash table of objects being fetched. */
|
||||||
|
@ -127,8 +126,7 @@ scheduling_algorithm_state *make_scheduling_algorithm_state(void) {
|
||||||
return algorithm_state;
|
return algorithm_state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_scheduling_algorithm_state(
|
void SchedulingAlgorithmState_free(SchedulingAlgorithmState *algorithm_state) {
|
||||||
scheduling_algorithm_state *algorithm_state) {
|
|
||||||
/* Free all of the tasks in the waiting queue. */
|
/* Free all of the tasks in the waiting queue. */
|
||||||
task_queue_entry *elt, *tmp1;
|
task_queue_entry *elt, *tmp1;
|
||||||
DL_FOREACH_SAFE(algorithm_state->waiting_task_queue, elt, tmp1) {
|
DL_FOREACH_SAFE(algorithm_state->waiting_task_queue, elt, tmp1) {
|
||||||
|
@ -143,7 +141,7 @@ void free_scheduling_algorithm_state(
|
||||||
free(elt);
|
free(elt);
|
||||||
}
|
}
|
||||||
/* Remove all of the remaining actors. */
|
/* Remove all of the remaining actors. */
|
||||||
local_actor_info *actor_entry, *tmp_actor_entry;
|
LocalActorInfo *actor_entry, *tmp_actor_entry;
|
||||||
HASH_ITER(hh, algorithm_state->local_actor_infos, actor_entry,
|
HASH_ITER(hh, algorithm_state->local_actor_infos, actor_entry,
|
||||||
tmp_actor_entry) {
|
tmp_actor_entry) {
|
||||||
/* We do not call HASH_DELETE here because it will be called inside of
|
/* We do not call HASH_DELETE here because it will be called inside of
|
||||||
|
@ -180,9 +178,9 @@ void free_scheduling_algorithm_state(
|
||||||
free(algorithm_state);
|
free(algorithm_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void provide_scheduler_info(local_scheduler_state *state,
|
void provide_scheduler_info(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_info *info) {
|
LocalSchedulerInfo *info) {
|
||||||
task_queue_entry *elt;
|
task_queue_entry *elt;
|
||||||
info->total_num_workers = utarray_len(state->workers);
|
info->total_num_workers = utarray_len(state->workers);
|
||||||
/* TODO(swang): Provide separate counts for tasks that are waiting for
|
/* TODO(swang): Provide separate counts for tasks that are waiting for
|
||||||
|
@ -203,7 +201,7 @@ void provide_scheduler_info(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the local_actor_info struct for an actor worker that this local
|
* Create the LocalActorInfo struct for an actor worker that this local
|
||||||
* scheduler is responsible for. For a given actor, this will either be done
|
* scheduler is responsible for. For a given actor, this will either be done
|
||||||
* when the first task for that actor arrives or when the worker running that
|
* when the first task for that actor arrives or when the worker running that
|
||||||
* actor connects to the local scheduler.
|
* actor connects to the local scheduler.
|
||||||
|
@ -217,11 +215,11 @@ void provide_scheduler_info(local_scheduler_state *state,
|
||||||
* has arrived), then this should be NULL.
|
* has arrived), then this should be NULL.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void create_actor(scheduling_algorithm_state *algorithm_state,
|
void create_actor(SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id,
|
ActorID actor_id,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
/* This will be freed when the actor is removed in remove_actor. */
|
/* This will be freed when the actor is removed in remove_actor. */
|
||||||
local_actor_info *entry = malloc(sizeof(local_actor_info));
|
LocalActorInfo *entry = malloc(sizeof(LocalActorInfo));
|
||||||
entry->actor_id = actor_id;
|
entry->actor_id = actor_id;
|
||||||
entry->task_counter = 0;
|
entry->task_counter = 0;
|
||||||
/* Initialize the doubly-linked list to NULL. */
|
/* Initialize the doubly-linked list to NULL. */
|
||||||
|
@ -234,13 +232,12 @@ void create_actor(scheduling_algorithm_state *algorithm_state,
|
||||||
/* Log some useful information about the actor that we created. */
|
/* Log some useful information about the actor that we created. */
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
LOG_DEBUG("Creating actor with ID %s.",
|
LOG_DEBUG("Creating actor with ID %s.",
|
||||||
object_id_to_string(actor_id, id_string, ID_STRING_SIZE));
|
ObjectID_to_string(actor_id, id_string, ID_STRING_SIZE));
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_actor(scheduling_algorithm_state *algorithm_state,
|
void remove_actor(SchedulingAlgorithmState *algorithm_state, ActorID actor_id) {
|
||||||
actor_id actor_id) {
|
LocalActorInfo *entry;
|
||||||
local_actor_info *entry;
|
|
||||||
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
||||||
entry);
|
entry);
|
||||||
/* Make sure the actor actually exists. */
|
/* Make sure the actor actually exists. */
|
||||||
|
@ -253,7 +250,7 @@ void remove_actor(scheduling_algorithm_state *algorithm_state,
|
||||||
DL_COUNT(entry->task_queue, elt, count);
|
DL_COUNT(entry->task_queue, elt, count);
|
||||||
if (count > 0) {
|
if (count > 0) {
|
||||||
LOG_WARN("Removing actor with ID %s and %d remaining tasks.",
|
LOG_WARN("Removing actor with ID %s and %d remaining tasks.",
|
||||||
object_id_to_string(actor_id, id_string, ID_STRING_SIZE), count);
|
ObjectID_to_string(actor_id, id_string, ID_STRING_SIZE), count);
|
||||||
}
|
}
|
||||||
UNUSED(id_string);
|
UNUSED(id_string);
|
||||||
|
|
||||||
|
@ -269,32 +266,32 @@ void remove_actor(scheduling_algorithm_state *algorithm_state,
|
||||||
free(entry);
|
free(entry);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_worker_connect(local_scheduler_state *state,
|
void handle_actor_worker_connect(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id,
|
ActorID actor_id,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
local_actor_info *entry;
|
LocalActorInfo *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
||||||
entry);
|
entry);
|
||||||
if (entry == NULL) {
|
if (entry == NULL) {
|
||||||
create_actor(algorithm_state, actor_id, worker);
|
create_actor(algorithm_state, actor_id, worker);
|
||||||
} else {
|
} else {
|
||||||
/* In this case, the local_actor_info struct was already been created by the
|
/* In this case, the LocalActorInfo struct was already been created by the
|
||||||
* first call to add_task_to_actor_queue. However, the worker field was not
|
* first call to add_task_to_actor_queue. However, the worker field was not
|
||||||
* filled out, so fill out the correct worker field now. */
|
* filled out, so fill out the correct worker field now. */
|
||||||
entry->worker = worker;
|
entry->worker = worker;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_worker_disconnect(local_scheduler_state *state,
|
void handle_actor_worker_disconnect(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id) {
|
ActorID actor_id) {
|
||||||
remove_actor(algorithm_state, actor_id);
|
remove_actor(algorithm_state, actor_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This will add a task to the task queue for an actor. If this is the first
|
* This will add a task to the task queue for an actor. If this is the first
|
||||||
* task being processed for this actor, it is possible that the local_actor_info
|
* task being processed for this actor, it is possible that the LocalActorInfo
|
||||||
* struct has not yet been created by create_worker (which happens when the
|
* struct has not yet been created by create_worker (which happens when the
|
||||||
* actor worker connects to the local scheduler), so in that case this method
|
* actor worker connects to the local scheduler), so in that case this method
|
||||||
* will call create_actor.
|
* will call create_actor.
|
||||||
|
@ -310,20 +307,20 @@ void handle_actor_worker_disconnect(local_scheduler_state *state,
|
||||||
* locally by a worker.
|
* locally by a worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void add_task_to_actor_queue(local_scheduler_state *state,
|
void add_task_to_actor_queue(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool from_global_scheduler) {
|
bool from_global_scheduler) {
|
||||||
actor_id actor_id = task_spec_actor_id(spec);
|
ActorID actor_id = task_spec_actor_id(spec);
|
||||||
char tmp[ID_STRING_SIZE];
|
char tmp[ID_STRING_SIZE];
|
||||||
object_id_to_string(actor_id, tmp, ID_STRING_SIZE);
|
ObjectID_to_string(actor_id, tmp, ID_STRING_SIZE);
|
||||||
DCHECK(!actor_ids_equal(actor_id, NIL_ACTOR_ID));
|
DCHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID));
|
||||||
/* Get the local actor entry for this actor. */
|
/* Get the local actor entry for this actor. */
|
||||||
local_actor_info *entry;
|
LocalActorInfo *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
||||||
entry);
|
entry);
|
||||||
|
|
||||||
/* Handle the case in which there is no local_actor_info struct yet. */
|
/* Handle the case in which there is no LocalActorInfo struct yet. */
|
||||||
if (entry == NULL) {
|
if (entry == NULL) {
|
||||||
/* Create the actor struct with a NULL worker because the worker struct has
|
/* Create the actor struct with a NULL worker because the worker struct has
|
||||||
* not been created yet. The correct worker struct will be inserted when the
|
* not been created yet. The correct worker struct will be inserted when the
|
||||||
|
@ -360,8 +357,8 @@ void add_task_to_actor_queue(local_scheduler_state *state,
|
||||||
|
|
||||||
/* Update the task table. */
|
/* Update the task table. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
task *task =
|
Task *task =
|
||||||
alloc_task(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db));
|
Task_alloc(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db));
|
||||||
if (from_global_scheduler) {
|
if (from_global_scheduler) {
|
||||||
/* If the task is from the global scheduler, it's already been added to
|
/* If the task is from the global scheduler, it's already been added to
|
||||||
* the task table, so just update the entry. */
|
* the task table, so just update the entry. */
|
||||||
|
@ -382,20 +379,20 @@ void add_task_to_actor_queue(local_scheduler_state *state,
|
||||||
* @param actor_id The ID of the actor corresponding to the worker.
|
* @param actor_id The ID of the actor corresponding to the worker.
|
||||||
* @return True if a task was dispatched to the actor and false otherwise.
|
* @return True if a task was dispatched to the actor and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool dispatch_actor_task(local_scheduler_state *state,
|
bool dispatch_actor_task(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id) {
|
ActorID actor_id) {
|
||||||
/* Make sure this worker actually is an actor. */
|
/* Make sure this worker actually is an actor. */
|
||||||
CHECK(!actor_ids_equal(actor_id, NIL_ACTOR_ID));
|
CHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID));
|
||||||
/* Make sure this actor belongs to this local scheduler. */
|
/* Make sure this actor belongs to this local scheduler. */
|
||||||
actor_map_entry *actor_entry;
|
actor_map_entry *actor_entry;
|
||||||
HASH_FIND(hh, state->actor_mapping, &actor_id, sizeof(actor_id), actor_entry);
|
HASH_FIND(hh, state->actor_mapping, &actor_id, sizeof(actor_id), actor_entry);
|
||||||
CHECK(actor_entry != NULL);
|
CHECK(actor_entry != NULL);
|
||||||
CHECK(db_client_ids_equal(actor_entry->local_scheduler_id,
|
CHECK(DBClientID_equal(actor_entry->local_scheduler_id,
|
||||||
get_db_client_id(state->db)));
|
get_db_client_id(state->db)));
|
||||||
|
|
||||||
/* Get the local actor entry for this actor. */
|
/* Get the local actor entry for this actor. */
|
||||||
local_actor_info *entry;
|
LocalActorInfo *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
||||||
entry);
|
entry);
|
||||||
CHECK(entry != NULL);
|
CHECK(entry != NULL);
|
||||||
|
@ -441,10 +438,10 @@ bool dispatch_actor_task(local_scheduler_state *state,
|
||||||
* @param obj_id The ID of the object that the task is dependent on.
|
* @param obj_id The ID of the object that the task is dependent on.
|
||||||
* @returns Void.
|
* @returns Void.
|
||||||
*/
|
*/
|
||||||
void fetch_missing_dependency(local_scheduler_state *state,
|
void fetch_missing_dependency(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_queue_entry *task_entry,
|
task_queue_entry *task_entry,
|
||||||
object_id obj_id) {
|
ObjectID obj_id) {
|
||||||
object_entry *entry;
|
object_entry *entry;
|
||||||
HASH_FIND(hh, algorithm_state->remote_objects, &obj_id, sizeof(obj_id),
|
HASH_FIND(hh, algorithm_state->remote_objects, &obj_id, sizeof(obj_id),
|
||||||
entry);
|
entry);
|
||||||
|
@ -478,15 +475,15 @@ void fetch_missing_dependency(local_scheduler_state *state,
|
||||||
* @param task_entry The task's queue entry.
|
* @param task_entry The task's queue entry.
|
||||||
* @returns Void.
|
* @returns Void.
|
||||||
*/
|
*/
|
||||||
void fetch_missing_dependencies(local_scheduler_state *state,
|
void fetch_missing_dependencies(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_queue_entry *task_entry) {
|
task_queue_entry *task_entry) {
|
||||||
task_spec *task = task_entry->spec;
|
task_spec *task = task_entry->spec;
|
||||||
int64_t num_args = task_num_args(task);
|
int64_t num_args = task_num_args(task);
|
||||||
int num_missing_dependencies = 0;
|
int num_missing_dependencies = 0;
|
||||||
for (int i = 0; i < num_args; ++i) {
|
for (int i = 0; i < num_args; ++i) {
|
||||||
if (task_arg_type(task, i) == ARG_BY_REF) {
|
if (task_arg_type(task, i) == ARG_BY_REF) {
|
||||||
object_id obj_id = task_arg_id(task, i);
|
ObjectID obj_id = task_arg_id(task, i);
|
||||||
object_entry *entry;
|
object_entry *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id),
|
HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id),
|
||||||
entry);
|
entry);
|
||||||
|
@ -510,11 +507,11 @@ void fetch_missing_dependencies(local_scheduler_state *state,
|
||||||
* task are present in the local object store, otherwise it returns
|
* task are present in the local object store, otherwise it returns
|
||||||
* false.
|
* false.
|
||||||
*/
|
*/
|
||||||
bool can_run(scheduling_algorithm_state *algorithm_state, task_spec *task) {
|
bool can_run(SchedulingAlgorithmState *algorithm_state, task_spec *task) {
|
||||||
int64_t num_args = task_num_args(task);
|
int64_t num_args = task_num_args(task);
|
||||||
for (int i = 0; i < num_args; ++i) {
|
for (int i = 0; i < num_args; ++i) {
|
||||||
if (task_arg_type(task, i) == ARG_BY_REF) {
|
if (task_arg_type(task, i) == ARG_BY_REF) {
|
||||||
object_id obj_id = task_arg_id(task, i);
|
ObjectID obj_id = task_arg_id(task, i);
|
||||||
object_entry *entry;
|
object_entry *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id),
|
HASH_FIND(hh, algorithm_state->local_objects, &obj_id, sizeof(obj_id),
|
||||||
entry);
|
entry);
|
||||||
|
@ -531,7 +528,7 @@ bool can_run(scheduling_algorithm_state *algorithm_state, task_spec *task) {
|
||||||
/* TODO(rkn): This method will need to be changed to call reconstruct. */
|
/* TODO(rkn): This method will need to be changed to call reconstruct. */
|
||||||
/* TODO(swang): This method is not covered by any valgrind tests. */
|
/* TODO(swang): This method is not covered by any valgrind tests. */
|
||||||
int fetch_object_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
int fetch_object_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
local_scheduler_state *state = context;
|
LocalSchedulerState *state = context;
|
||||||
/* Only try the fetches if we are connected to the object store manager. */
|
/* Only try the fetches if we are connected to the object store manager. */
|
||||||
if (!plasma_manager_is_connected(state->plasma_conn)) {
|
if (!plasma_manager_is_connected(state->plasma_conn)) {
|
||||||
LOG_INFO("Local scheduler is not connected to a object store manager");
|
LOG_INFO("Local scheduler is not connected to a object store manager");
|
||||||
|
@ -540,7 +537,7 @@ int fetch_object_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
|
|
||||||
/* Allocate a buffer to hold all the object IDs for active fetch requests. */
|
/* Allocate a buffer to hold all the object IDs for active fetch requests. */
|
||||||
int num_object_ids = HASH_COUNT(state->algorithm_state->remote_objects);
|
int num_object_ids = HASH_COUNT(state->algorithm_state->remote_objects);
|
||||||
object_id *object_ids = malloc(num_object_ids * sizeof(object_id));
|
ObjectID *object_ids = malloc(num_object_ids * sizeof(ObjectID));
|
||||||
|
|
||||||
/* Fill out the request with the object IDs for active fetches. */
|
/* Fill out the request with the object IDs for active fetches. */
|
||||||
object_entry *fetch_request, *tmp;
|
object_entry *fetch_request, *tmp;
|
||||||
|
@ -564,8 +561,8 @@ int fetch_object_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
* @param algorithm_state The scheduling algorithm state.
|
* @param algorithm_state The scheduling algorithm state.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void dispatch_tasks(local_scheduler_state *state,
|
void dispatch_tasks(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state) {
|
SchedulingAlgorithmState *algorithm_state) {
|
||||||
task_queue_entry *elt, *tmp;
|
task_queue_entry *elt, *tmp;
|
||||||
|
|
||||||
/* Assign as many tasks as we can, while there are workers available. */
|
/* Assign as many tasks as we can, while there are workers available. */
|
||||||
|
@ -612,7 +609,7 @@ void dispatch_tasks(local_scheduler_state *state,
|
||||||
/* Dispatch this task to an available worker and dequeue the task. */
|
/* Dispatch this task to an available worker and dequeue the task. */
|
||||||
LOG_DEBUG("Dispatching task");
|
LOG_DEBUG("Dispatching task");
|
||||||
/* Get the last available worker in the available worker queue. */
|
/* Get the last available worker in the available worker queue. */
|
||||||
local_scheduler_client **worker = (local_scheduler_client **) utarray_back(
|
LocalSchedulerClient **worker = (LocalSchedulerClient **) utarray_back(
|
||||||
algorithm_state->available_workers);
|
algorithm_state->available_workers);
|
||||||
/* Tell the available worker to execute the task. */
|
/* Tell the available worker to execute the task. */
|
||||||
assign_task_to_worker(state, elt->spec, *worker);
|
assign_task_to_worker(state, elt->spec, *worker);
|
||||||
|
@ -642,7 +639,7 @@ void dispatch_tasks(local_scheduler_state *state,
|
||||||
* scheduler. If false, the task was submitted by a worker.
|
* scheduler. If false, the task was submitted by a worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
task_queue_entry *queue_task(local_scheduler_state *state,
|
task_queue_entry *queue_task(LocalSchedulerState *state,
|
||||||
task_queue_entry **task_queue,
|
task_queue_entry **task_queue,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool from_global_scheduler) {
|
bool from_global_scheduler) {
|
||||||
|
@ -656,8 +653,8 @@ task_queue_entry *queue_task(local_scheduler_state *state,
|
||||||
/* The task has been added to a local scheduler queue. Write the entry in the
|
/* 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. */
|
* task table to notify others that we have queued it. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
task *task =
|
Task *task =
|
||||||
alloc_task(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db));
|
Task_alloc(spec, TASK_STATUS_QUEUED, get_db_client_id(state->db));
|
||||||
if (from_global_scheduler) {
|
if (from_global_scheduler) {
|
||||||
/* If the task is from the global scheduler, it's already been added to
|
/* If the task is from the global scheduler, it's already been added to
|
||||||
* the task table, so just update the entry. */
|
* the task table, so just update the entry. */
|
||||||
|
@ -685,8 +682,8 @@ task_queue_entry *queue_task(local_scheduler_state *state,
|
||||||
* scheduler. If false, the task was submitted by a worker.
|
* scheduler. If false, the task was submitted by a worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void queue_waiting_task(local_scheduler_state *state,
|
void queue_waiting_task(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool from_global_scheduler) {
|
bool from_global_scheduler) {
|
||||||
LOG_DEBUG("Queueing task in waiting queue");
|
LOG_DEBUG("Queueing task in waiting queue");
|
||||||
|
@ -708,8 +705,8 @@ void queue_waiting_task(local_scheduler_state *state,
|
||||||
* scheduler. If false, the task was submitted by a worker.
|
* scheduler. If false, the task was submitted by a worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void queue_dispatch_task(local_scheduler_state *state,
|
void queue_dispatch_task(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool from_global_scheduler) {
|
bool from_global_scheduler) {
|
||||||
LOG_DEBUG("Queueing task in dispatch queue");
|
LOG_DEBUG("Queueing task in dispatch queue");
|
||||||
|
@ -729,8 +726,8 @@ void queue_dispatch_task(local_scheduler_state *state,
|
||||||
* scheduler. If false, the task was submitted by a worker.
|
* scheduler. If false, the task was submitted by a worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void queue_task_locally(local_scheduler_state *state,
|
void queue_task_locally(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool from_global_scheduler) {
|
bool from_global_scheduler) {
|
||||||
if (can_run(algorithm_state, spec)) {
|
if (can_run(algorithm_state, spec)) {
|
||||||
|
@ -752,17 +749,17 @@ void queue_task_locally(local_scheduler_state *state,
|
||||||
* @param local_scheduler_id The ID of the local scheduler to give the task to.
|
* @param local_scheduler_id The ID of the local scheduler to give the task to.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void give_task_to_local_scheduler(local_scheduler_state *state,
|
void give_task_to_local_scheduler(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
db_client_id local_scheduler_id) {
|
DBClientID local_scheduler_id) {
|
||||||
if (db_client_ids_equal(local_scheduler_id, get_db_client_id(state->db))) {
|
if (DBClientID_equal(local_scheduler_id, get_db_client_id(state->db))) {
|
||||||
LOG_WARN("Local scheduler is trying to assign a task to itself.");
|
LOG_WARN("Local scheduler is trying to assign a task to itself.");
|
||||||
}
|
}
|
||||||
CHECK(state->db != NULL);
|
CHECK(state->db != NULL);
|
||||||
/* Assign the task to the relevant local scheduler. */
|
/* Assign the task to the relevant local scheduler. */
|
||||||
DCHECK(state->config.global_scheduler_exists);
|
DCHECK(state->config.global_scheduler_exists);
|
||||||
task *task = alloc_task(spec, TASK_STATUS_SCHEDULED, local_scheduler_id);
|
Task *task = Task_alloc(spec, TASK_STATUS_SCHEDULED, local_scheduler_id);
|
||||||
task_table_add_task(state->db, task, NULL, NULL, NULL);
|
task_table_add_task(state->db, task, NULL, NULL, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -774,8 +771,8 @@ void give_task_to_local_scheduler(local_scheduler_state *state,
|
||||||
* @param spec The task specification to schedule.
|
* @param spec The task specification to schedule.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void give_task_to_global_scheduler(local_scheduler_state *state,
|
void give_task_to_global_scheduler(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
if (state->db == NULL || !state->config.global_scheduler_exists) {
|
if (state->db == NULL || !state->config.global_scheduler_exists) {
|
||||||
/* A global scheduler is not available, so queue the task locally. */
|
/* A global scheduler is not available, so queue the task locally. */
|
||||||
|
@ -784,12 +781,12 @@ void give_task_to_global_scheduler(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
/* Pass on the task to the global scheduler. */
|
/* Pass on the task to the global scheduler. */
|
||||||
DCHECK(state->config.global_scheduler_exists);
|
DCHECK(state->config.global_scheduler_exists);
|
||||||
task *task = alloc_task(spec, TASK_STATUS_WAITING, NIL_ID);
|
Task *task = Task_alloc(spec, TASK_STATUS_WAITING, NIL_ID);
|
||||||
DCHECK(state->db != NULL);
|
DCHECK(state->db != NULL);
|
||||||
task_table_add_task(state->db, task, NULL, NULL, NULL);
|
task_table_add_task(state->db, task, NULL, NULL, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool resource_constraints_satisfied(local_scheduler_state *state,
|
bool resource_constraints_satisfied(LocalSchedulerState *state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
/* At the local scheduler, if required resource vector exceeds either static
|
/* At the local scheduler, if required resource vector exceeds either static
|
||||||
* or dynamic resource vector, the resource constraint is not satisfied. */
|
* or dynamic resource vector, the resource constraint is not satisfied. */
|
||||||
|
@ -803,8 +800,8 @@ bool resource_constraints_satisfied(local_scheduler_state *state,
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_task_submitted(local_scheduler_state *state,
|
void handle_task_submitted(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
/* TODO(atumanov): if static is satisfied and local objects ready, but dynamic
|
/* TODO(atumanov): if static is satisfied and local objects ready, but dynamic
|
||||||
* resource is currently unavailable, then consider queueing task locally and
|
* resource is currently unavailable, then consider queueing task locally and
|
||||||
|
@ -827,11 +824,11 @@ void handle_task_submitted(local_scheduler_state *state,
|
||||||
dispatch_tasks(state, algorithm_state);
|
dispatch_tasks(state, algorithm_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_task_submitted(local_scheduler_state *state,
|
void handle_actor_task_submitted(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
actor_id actor_id = task_spec_actor_id(spec);
|
ActorID actor_id = task_spec_actor_id(spec);
|
||||||
CHECK(!actor_ids_equal(actor_id, NIL_ACTOR_ID));
|
CHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID));
|
||||||
|
|
||||||
/* Find the local scheduler responsible for this actor. */
|
/* Find the local scheduler responsible for this actor. */
|
||||||
actor_map_entry *entry;
|
actor_map_entry *entry;
|
||||||
|
@ -846,8 +843,8 @@ void handle_actor_task_submitted(local_scheduler_state *state,
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (db_client_ids_equal(entry->local_scheduler_id,
|
if (DBClientID_equal(entry->local_scheduler_id,
|
||||||
get_db_client_id(state->db))) {
|
get_db_client_id(state->db))) {
|
||||||
/* This local scheduler is responsible for the actor, so handle the task
|
/* This local scheduler is responsible for the actor, so handle the task
|
||||||
* locally. */
|
* locally. */
|
||||||
add_task_to_actor_queue(state, algorithm_state, spec, false);
|
add_task_to_actor_queue(state, algorithm_state, spec, false);
|
||||||
|
@ -862,9 +859,9 @@ void handle_actor_task_submitted(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_creation_notification(
|
void handle_actor_creation_notification(
|
||||||
local_scheduler_state *state,
|
LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id) {
|
ActorID actor_id) {
|
||||||
int num_cached_actor_tasks =
|
int num_cached_actor_tasks =
|
||||||
utarray_len(algorithm_state->cached_submitted_actor_tasks);
|
utarray_len(algorithm_state->cached_submitted_actor_tasks);
|
||||||
for (int i = 0; i < num_cached_actor_tasks; ++i) {
|
for (int i = 0; i < num_cached_actor_tasks; ++i) {
|
||||||
|
@ -880,8 +877,8 @@ void handle_actor_creation_notification(
|
||||||
num_cached_actor_tasks);
|
num_cached_actor_tasks);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_task_scheduled(local_scheduler_state *state,
|
void handle_task_scheduled(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
/* This callback handles tasks that were assigned to this local scheduler by
|
/* 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 global scheduler, so we can safely assert that there is a connection to
|
||||||
|
@ -893,8 +890,8 @@ void handle_task_scheduled(local_scheduler_state *state,
|
||||||
dispatch_tasks(state, algorithm_state);
|
dispatch_tasks(state, algorithm_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_task_scheduled(local_scheduler_state *state,
|
void handle_actor_task_scheduled(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec) {
|
task_spec *spec) {
|
||||||
/* This callback handles tasks that were assigned to this local scheduler by
|
/* 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
|
* the global scheduler or by other workers, so we can safely assert that
|
||||||
|
@ -903,8 +900,8 @@ void handle_actor_task_scheduled(local_scheduler_state *state,
|
||||||
DCHECK(state->config.global_scheduler_exists);
|
DCHECK(state->config.global_scheduler_exists);
|
||||||
/* Check that the task is meant to run on an actor that this local scheduler
|
/* Check that the task is meant to run on an actor that this local scheduler
|
||||||
* is responsible for. */
|
* is responsible for. */
|
||||||
actor_id actor_id = task_spec_actor_id(spec);
|
ActorID actor_id = task_spec_actor_id(spec);
|
||||||
DCHECK(!actor_ids_equal(actor_id, NIL_ACTOR_ID));
|
DCHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID));
|
||||||
actor_map_entry *entry;
|
actor_map_entry *entry;
|
||||||
HASH_FIND(hh, state->actor_mapping, &actor_id, sizeof(actor_id), entry);
|
HASH_FIND(hh, state->actor_mapping, &actor_id, sizeof(actor_id), entry);
|
||||||
if (entry != NULL) {
|
if (entry != NULL) {
|
||||||
|
@ -913,8 +910,8 @@ void handle_actor_task_scheduled(local_scheduler_state *state,
|
||||||
* local scheduler has not yet processed the notification about the actor
|
* local scheduler has not yet processed the notification about the actor
|
||||||
* creation. This may be possible though should be very uncommon. If it does
|
* creation. This may be possible though should be very uncommon. If it does
|
||||||
* happen, it's ok. */
|
* happen, it's ok. */
|
||||||
DCHECK(db_client_ids_equal(entry->local_scheduler_id,
|
DCHECK(DBClientID_equal(entry->local_scheduler_id,
|
||||||
get_db_client_id(state->db)));
|
get_db_client_id(state->db)));
|
||||||
} else {
|
} else {
|
||||||
LOG_INFO(
|
LOG_INFO(
|
||||||
"handle_actor_task_scheduled called on local scheduler but the "
|
"handle_actor_task_scheduled called on local scheduler but the "
|
||||||
|
@ -925,28 +922,28 @@ void handle_actor_task_scheduled(local_scheduler_state *state,
|
||||||
dispatch_actor_task(state, algorithm_state, actor_id);
|
dispatch_actor_task(state, algorithm_state, actor_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_worker_available(local_scheduler_state *state,
|
void handle_worker_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
CHECK(worker->task_in_progress == NULL);
|
CHECK(worker->task_in_progress == NULL);
|
||||||
/* Check that the worker isn't in the pool of available workers. */
|
/* Check that the worker isn't in the pool of available workers. */
|
||||||
for (local_scheduler_client **p = (local_scheduler_client **) utarray_front(
|
for (LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_front(
|
||||||
algorithm_state->available_workers);
|
algorithm_state->available_workers);
|
||||||
p != NULL; p = (local_scheduler_client **) utarray_next(
|
p != NULL; p = (LocalSchedulerClient **) utarray_next(
|
||||||
algorithm_state->available_workers, p)) {
|
algorithm_state->available_workers, p)) {
|
||||||
DCHECK(*p != worker);
|
DCHECK(*p != worker);
|
||||||
}
|
}
|
||||||
/* Check that the worker isn't in the list of blocked workers. */
|
/* Check that the worker isn't in the list of blocked workers. */
|
||||||
for (local_scheduler_client **p = (local_scheduler_client **) utarray_front(
|
for (LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_front(
|
||||||
algorithm_state->blocked_workers);
|
algorithm_state->blocked_workers);
|
||||||
p != NULL; p = (local_scheduler_client **) utarray_next(
|
p != NULL; p = (LocalSchedulerClient **) utarray_next(
|
||||||
algorithm_state->blocked_workers, p)) {
|
algorithm_state->blocked_workers, p)) {
|
||||||
DCHECK(*p != worker);
|
DCHECK(*p != worker);
|
||||||
}
|
}
|
||||||
/* If the worker was executing a task, it must have finished, so remove it
|
/* If the worker was executing a task, it must have finished, so remove it
|
||||||
* from the list of executing workers. */
|
* from the list of executing workers. */
|
||||||
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->executing_workers, i);
|
algorithm_state->executing_workers, i);
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
utarray_erase(algorithm_state->executing_workers, i, 1);
|
utarray_erase(algorithm_state->executing_workers, i, 1);
|
||||||
|
@ -955,7 +952,7 @@ void handle_worker_available(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
/* Check that we actually erased the worker. */
|
/* Check that we actually erased the worker. */
|
||||||
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->executing_workers, i);
|
algorithm_state->executing_workers, i);
|
||||||
DCHECK(*p != worker);
|
DCHECK(*p != worker);
|
||||||
}
|
}
|
||||||
|
@ -968,9 +965,9 @@ void handle_worker_available(local_scheduler_state *state,
|
||||||
dispatch_tasks(state, algorithm_state);
|
dispatch_tasks(state, algorithm_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_worker_removed(local_scheduler_state *state,
|
void handle_worker_removed(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
/* Make sure that we remove the worker at most once. */
|
/* Make sure that we remove the worker at most once. */
|
||||||
bool removed = false;
|
bool removed = false;
|
||||||
int64_t num_workers;
|
int64_t num_workers;
|
||||||
|
@ -978,7 +975,7 @@ void handle_worker_removed(local_scheduler_state *state,
|
||||||
/* Remove the worker from available workers, if it's there. */
|
/* Remove the worker from available workers, if it's there. */
|
||||||
num_workers = utarray_len(algorithm_state->available_workers);
|
num_workers = utarray_len(algorithm_state->available_workers);
|
||||||
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->available_workers, i);
|
algorithm_state->available_workers, i);
|
||||||
DCHECK(!((*p == worker) && removed));
|
DCHECK(!((*p == worker) && removed));
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
|
@ -990,7 +987,7 @@ void handle_worker_removed(local_scheduler_state *state,
|
||||||
/* Remove the worker from executing workers, if it's there. */
|
/* Remove the worker from executing workers, if it's there. */
|
||||||
num_workers = utarray_len(algorithm_state->executing_workers);
|
num_workers = utarray_len(algorithm_state->executing_workers);
|
||||||
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->executing_workers, i);
|
algorithm_state->executing_workers, i);
|
||||||
DCHECK(!((*p == worker) && removed));
|
DCHECK(!((*p == worker) && removed));
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
|
@ -1002,7 +999,7 @@ void handle_worker_removed(local_scheduler_state *state,
|
||||||
/* Remove the worker from blocked workers, if it's there. */
|
/* Remove the worker from blocked workers, if it's there. */
|
||||||
num_workers = utarray_len(algorithm_state->blocked_workers);
|
num_workers = utarray_len(algorithm_state->blocked_workers);
|
||||||
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
for (int64_t i = num_workers - 1; i >= 0; --i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->blocked_workers, i);
|
algorithm_state->blocked_workers, i);
|
||||||
DCHECK(!((*p == worker) && removed));
|
DCHECK(!((*p == worker) && removed));
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
|
@ -1012,13 +1009,13 @@ void handle_worker_removed(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_actor_worker_available(local_scheduler_state *state,
|
void handle_actor_worker_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
actor_id actor_id = worker->actor_id;
|
ActorID actor_id = worker->actor_id;
|
||||||
CHECK(!actor_ids_equal(actor_id, NIL_ACTOR_ID));
|
CHECK(!ActorID_equal(actor_id, NIL_ACTOR_ID));
|
||||||
/* Get the actor info for this worker. */
|
/* Get the actor info for this worker. */
|
||||||
local_actor_info *entry;
|
LocalActorInfo *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
HASH_FIND(hh, algorithm_state->local_actor_infos, &actor_id, sizeof(actor_id),
|
||||||
entry);
|
entry);
|
||||||
CHECK(entry != NULL);
|
CHECK(entry != NULL);
|
||||||
|
@ -1029,29 +1026,28 @@ void handle_actor_worker_available(local_scheduler_state *state,
|
||||||
dispatch_actor_task(state, algorithm_state, actor_id);
|
dispatch_actor_task(state, algorithm_state, actor_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_worker_blocked(local_scheduler_state *state,
|
void handle_worker_blocked(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
/* Find the worker in the list of executing workers. */
|
/* Find the worker in the list of executing workers. */
|
||||||
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
for (int i = 0; i < utarray_len(algorithm_state->executing_workers); ++i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->executing_workers, i);
|
algorithm_state->executing_workers, i);
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
/* Remove the worker from the list of executing workers. */
|
/* Remove the worker from the list of executing workers. */
|
||||||
utarray_erase(algorithm_state->executing_workers, i, 1);
|
utarray_erase(algorithm_state->executing_workers, i, 1);
|
||||||
|
|
||||||
/* Check that the worker isn't in the list of blocked workers. */
|
/* Check that the worker isn't in the list of blocked workers. */
|
||||||
for (local_scheduler_client **q =
|
for (LocalSchedulerClient **q = (LocalSchedulerClient **) utarray_front(
|
||||||
(local_scheduler_client **) utarray_front(
|
algorithm_state->blocked_workers);
|
||||||
algorithm_state->blocked_workers);
|
q != NULL; q = (LocalSchedulerClient **) utarray_next(
|
||||||
q != NULL; q = (local_scheduler_client **) utarray_next(
|
|
||||||
algorithm_state->blocked_workers, q)) {
|
algorithm_state->blocked_workers, q)) {
|
||||||
DCHECK(*q != worker);
|
DCHECK(*q != worker);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Return the resources that the blocked worker was using. */
|
/* Return the resources that the blocked worker was using. */
|
||||||
CHECK(worker->task_in_progress != NULL);
|
CHECK(worker->task_in_progress != NULL);
|
||||||
task_spec *spec = task_task_spec(worker->task_in_progress);
|
task_spec *spec = Task_task_spec(worker->task_in_progress);
|
||||||
update_dynamic_resources(state, spec, true);
|
update_dynamic_resources(state, spec, true);
|
||||||
/* Add the worker to the list of blocked workers. */
|
/* Add the worker to the list of blocked workers. */
|
||||||
worker->is_blocked = true;
|
worker->is_blocked = true;
|
||||||
|
@ -1068,22 +1064,21 @@ void handle_worker_blocked(local_scheduler_state *state,
|
||||||
"workers.");
|
"workers.");
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_worker_unblocked(local_scheduler_state *state,
|
void handle_worker_unblocked(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
/* Find the worker in the list of blocked workers. */
|
/* Find the worker in the list of blocked workers. */
|
||||||
for (int i = 0; i < utarray_len(algorithm_state->blocked_workers); ++i) {
|
for (int i = 0; i < utarray_len(algorithm_state->blocked_workers); ++i) {
|
||||||
local_scheduler_client **p = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **p = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
algorithm_state->blocked_workers, i);
|
algorithm_state->blocked_workers, i);
|
||||||
if (*p == worker) {
|
if (*p == worker) {
|
||||||
/* Remove the worker from the list of blocked workers. */
|
/* Remove the worker from the list of blocked workers. */
|
||||||
utarray_erase(algorithm_state->blocked_workers, i, 1);
|
utarray_erase(algorithm_state->blocked_workers, i, 1);
|
||||||
|
|
||||||
/* Check that the worker isn't in the list of executing workers. */
|
/* Check that the worker isn't in the list of executing workers. */
|
||||||
for (local_scheduler_client **q =
|
for (LocalSchedulerClient **q = (LocalSchedulerClient **) utarray_front(
|
||||||
(local_scheduler_client **) utarray_front(
|
algorithm_state->executing_workers);
|
||||||
algorithm_state->executing_workers);
|
q != NULL; q = (LocalSchedulerClient **) utarray_next(
|
||||||
q != NULL; q = (local_scheduler_client **) utarray_next(
|
|
||||||
algorithm_state->executing_workers, q)) {
|
algorithm_state->executing_workers, q)) {
|
||||||
DCHECK(*q != worker);
|
DCHECK(*q != worker);
|
||||||
}
|
}
|
||||||
|
@ -1094,7 +1089,7 @@ void handle_worker_unblocked(local_scheduler_state *state,
|
||||||
* fixed by having blocked workers explicitly yield and wait to be given
|
* fixed by having blocked workers explicitly yield and wait to be given
|
||||||
* back resources before continuing execution. */
|
* back resources before continuing execution. */
|
||||||
CHECK(worker->task_in_progress != NULL);
|
CHECK(worker->task_in_progress != NULL);
|
||||||
task_spec *spec = task_task_spec(worker->task_in_progress);
|
task_spec *spec = Task_task_spec(worker->task_in_progress);
|
||||||
update_dynamic_resources(state, spec, false);
|
update_dynamic_resources(state, spec, false);
|
||||||
/* Add the worker to the list of executing workers. */
|
/* Add the worker to the list of executing workers. */
|
||||||
worker->is_blocked = false;
|
worker->is_blocked = false;
|
||||||
|
@ -1111,9 +1106,9 @@ void handle_worker_unblocked(local_scheduler_state *state,
|
||||||
"workers.");
|
"workers.");
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_object_available(local_scheduler_state *state,
|
void handle_object_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
/* Get the entry for this object from the active fetch request, or allocate
|
/* Get the entry for this object from the active fetch request, or allocate
|
||||||
* one if needed. */
|
* one if needed. */
|
||||||
object_entry *entry;
|
object_entry *entry;
|
||||||
|
@ -1157,10 +1152,10 @@ void handle_object_available(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void handle_object_removed(local_scheduler_state *state,
|
void handle_object_removed(LocalSchedulerState *state,
|
||||||
object_id removed_object_id) {
|
ObjectID removed_object_id) {
|
||||||
/* Remove the object from the set of locally available objects. */
|
/* Remove the object from the set of locally available objects. */
|
||||||
scheduling_algorithm_state *algorithm_state = state->algorithm_state;
|
SchedulingAlgorithmState *algorithm_state = state->algorithm_state;
|
||||||
object_entry *entry;
|
object_entry *entry;
|
||||||
HASH_FIND(hh, algorithm_state->local_objects, &removed_object_id,
|
HASH_FIND(hh, algorithm_state->local_objects, &removed_object_id,
|
||||||
sizeof(removed_object_id), entry);
|
sizeof(removed_object_id), entry);
|
||||||
|
@ -1180,8 +1175,8 @@ void handle_object_removed(local_scheduler_state *state,
|
||||||
int64_t num_args = task_num_args(task);
|
int64_t num_args = task_num_args(task);
|
||||||
for (int i = 0; i < num_args; ++i) {
|
for (int i = 0; i < num_args; ++i) {
|
||||||
if (task_arg_type(task, i) == ARG_BY_REF) {
|
if (task_arg_type(task, i) == ARG_BY_REF) {
|
||||||
object_id arg_id = task_arg_id(task, i);
|
ObjectID arg_id = task_arg_id(task, i);
|
||||||
if (object_ids_equal(arg_id, removed_object_id)) {
|
if (ObjectID_equal(arg_id, removed_object_id)) {
|
||||||
fetch_missing_dependency(state, algorithm_state, elt,
|
fetch_missing_dependency(state, algorithm_state, elt,
|
||||||
removed_object_id);
|
removed_object_id);
|
||||||
}
|
}
|
||||||
|
@ -1195,8 +1190,8 @@ void handle_object_removed(local_scheduler_state *state,
|
||||||
int64_t num_args = task_num_args(task);
|
int64_t num_args = task_num_args(task);
|
||||||
for (int i = 0; i < num_args; ++i) {
|
for (int i = 0; i < num_args; ++i) {
|
||||||
if (task_arg_type(task, i) == ARG_BY_REF) {
|
if (task_arg_type(task, i) == ARG_BY_REF) {
|
||||||
object_id arg_id = task_arg_id(task, i);
|
ObjectID arg_id = task_arg_id(task, i);
|
||||||
if (object_ids_equal(arg_id, removed_object_id)) {
|
if (ObjectID_equal(arg_id, removed_object_id)) {
|
||||||
LOG_DEBUG("Moved task from dispatch queue back to waiting queue");
|
LOG_DEBUG("Moved task from dispatch queue back to waiting queue");
|
||||||
DL_DELETE(algorithm_state->dispatch_task_queue, elt);
|
DL_DELETE(algorithm_state->dispatch_task_queue, elt);
|
||||||
DL_APPEND(algorithm_state->waiting_task_queue, elt);
|
DL_APPEND(algorithm_state->waiting_task_queue, elt);
|
||||||
|
@ -1208,14 +1203,14 @@ void handle_object_removed(local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
int num_waiting_tasks(scheduling_algorithm_state *algorithm_state) {
|
int num_waiting_tasks(SchedulingAlgorithmState *algorithm_state) {
|
||||||
task_queue_entry *elt;
|
task_queue_entry *elt;
|
||||||
int count;
|
int count;
|
||||||
DL_COUNT(algorithm_state->waiting_task_queue, elt, count);
|
DL_COUNT(algorithm_state->waiting_task_queue, elt, count);
|
||||||
return count;
|
return count;
|
||||||
}
|
}
|
||||||
|
|
||||||
int num_dispatch_tasks(scheduling_algorithm_state *algorithm_state) {
|
int num_dispatch_tasks(SchedulingAlgorithmState *algorithm_state) {
|
||||||
task_queue_entry *elt;
|
task_queue_entry *elt;
|
||||||
int count;
|
int count;
|
||||||
DL_COUNT(algorithm_state->dispatch_task_queue, elt, count);
|
DL_COUNT(algorithm_state->dispatch_task_queue, elt, count);
|
||||||
|
@ -1223,7 +1218,7 @@ int num_dispatch_tasks(scheduling_algorithm_state *algorithm_state) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void print_worker_info(const char *message,
|
void print_worker_info(const char *message,
|
||||||
scheduling_algorithm_state *algorithm_state) {
|
SchedulingAlgorithmState *algorithm_state) {
|
||||||
LOG_DEBUG("%s: %d available, %d executing, %d blocked", message,
|
LOG_DEBUG("%s: %d available, %d executing, %d blocked", message,
|
||||||
utarray_len(algorithm_state->available_workers),
|
utarray_len(algorithm_state->available_workers),
|
||||||
utarray_len(algorithm_state->executing_workers),
|
utarray_len(algorithm_state->executing_workers),
|
||||||
|
|
|
@ -23,7 +23,7 @@
|
||||||
*
|
*
|
||||||
* @return State managed by the scheduling algorithm.
|
* @return State managed by the scheduling algorithm.
|
||||||
*/
|
*/
|
||||||
scheduling_algorithm_state *make_scheduling_algorithm_state(void);
|
SchedulingAlgorithmState *SchedulingAlgorithmState_init(void);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Free the scheduler state.
|
* Free the scheduler state.
|
||||||
|
@ -31,15 +31,14 @@ scheduling_algorithm_state *make_scheduling_algorithm_state(void);
|
||||||
* @param algorithm_state State maintained by the scheduling algorithm.
|
* @param algorithm_state State maintained by the scheduling algorithm.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void free_scheduling_algorithm_state(
|
void SchedulingAlgorithmState_free(SchedulingAlgorithmState *algorithm_state);
|
||||||
scheduling_algorithm_state *algorithm_state);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
void provide_scheduler_info(local_scheduler_state *state,
|
void provide_scheduler_info(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_info *info);
|
LocalSchedulerInfo *info);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function will be called when a new task is submitted by a worker for
|
* This function will be called when a new task is submitted by a worker for
|
||||||
|
@ -57,8 +56,8 @@ void provide_scheduler_info(local_scheduler_state *state,
|
||||||
* @param task Task that is submitted by the worker.
|
* @param task Task that is submitted by the worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_task_submitted(local_scheduler_state *state,
|
void handle_task_submitted(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec);
|
task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -70,8 +69,8 @@ void handle_task_submitted(local_scheduler_state *state,
|
||||||
* @param task Task that is submitted by the worker.
|
* @param task Task that is submitted by the worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_task_submitted(local_scheduler_state *state,
|
void handle_actor_task_submitted(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec);
|
task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -85,9 +84,9 @@ void handle_actor_task_submitted(local_scheduler_state *state,
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_creation_notification(
|
void handle_actor_creation_notification(
|
||||||
local_scheduler_state *state,
|
LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id);
|
ActorID actor_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function will be called when a task is assigned by the global scheduler
|
* This function will be called when a task is assigned by the global scheduler
|
||||||
|
@ -98,8 +97,8 @@ void handle_actor_creation_notification(
|
||||||
* @param task Task that is assigned by the global scheduler.
|
* @param task Task that is assigned by the global scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_task_scheduled(local_scheduler_state *state,
|
void handle_task_scheduled(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec);
|
task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -112,8 +111,8 @@ void handle_task_scheduled(local_scheduler_state *state,
|
||||||
* @param task Task that is assigned by the global scheduler.
|
* @param task Task that is assigned by the global scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_task_scheduled(local_scheduler_state *state,
|
void handle_actor_task_scheduled(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
task_spec *spec);
|
task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -125,9 +124,9 @@ void handle_actor_task_scheduled(local_scheduler_state *state,
|
||||||
* @param object_id ID of the object that became available.
|
* @param object_id ID of the object that became available.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_object_available(local_scheduler_state *state,
|
void handle_object_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function is called if an object is removed from the local plasma store.
|
* This function is called if an object is removed from the local plasma store.
|
||||||
|
@ -136,7 +135,7 @@ void handle_object_available(local_scheduler_state *state,
|
||||||
* @param object_id ID of the object that was removed.
|
* @param object_id ID of the object that was removed.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_object_removed(local_scheduler_state *state, object_id object_id);
|
void handle_object_removed(LocalSchedulerState *state, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function is called when a new worker becomes available.
|
* This function is called when a new worker becomes available.
|
||||||
|
@ -146,9 +145,9 @@ void handle_object_removed(local_scheduler_state *state, object_id object_id);
|
||||||
* @param worker The worker that is available.
|
* @param worker The worker that is available.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_worker_available(local_scheduler_state *state,
|
void handle_worker_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function is called when a worker is removed.
|
* This function is called when a worker is removed.
|
||||||
|
@ -158,9 +157,9 @@ void handle_worker_available(local_scheduler_state *state,
|
||||||
* @param worker The worker that is removed.
|
* @param worker The worker that is removed.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_worker_removed(local_scheduler_state *state,
|
void handle_worker_removed(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This version of handle_worker_available is called whenever the worker that is
|
* This version of handle_worker_available is called whenever the worker that is
|
||||||
|
@ -171,9 +170,9 @@ void handle_worker_removed(local_scheduler_state *state,
|
||||||
* @param wi Information about the worker that is available.
|
* @param wi Information about the worker that is available.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_worker_available(local_scheduler_state *state,
|
void handle_actor_worker_available(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle the fact that a new worker is available for running an actor.
|
* Handle the fact that a new worker is available for running an actor.
|
||||||
|
@ -184,10 +183,10 @@ void handle_actor_worker_available(local_scheduler_state *state,
|
||||||
* @param worker The worker that was connected.
|
* @param worker The worker that was connected.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_worker_connect(local_scheduler_state *state,
|
void handle_actor_worker_connect(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id,
|
ActorID actor_id,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle the fact that a worker running an actor has disconnected.
|
* Handle the fact that a worker running an actor has disconnected.
|
||||||
|
@ -197,9 +196,9 @@ void handle_actor_worker_connect(local_scheduler_state *state,
|
||||||
* @param actor_id The ID of the actor running on the worker.
|
* @param actor_id The ID of the actor running on the worker.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_worker_disconnect(local_scheduler_state *state,
|
void handle_actor_worker_disconnect(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
actor_id actor_id);
|
ActorID actor_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function is called when a worker that was executing a task becomes
|
* This function is called when a worker that was executing a task becomes
|
||||||
|
@ -210,9 +209,9 @@ void handle_actor_worker_disconnect(local_scheduler_state *state,
|
||||||
* @param worker The worker that is blocked.
|
* @param worker The worker that is blocked.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_worker_blocked(local_scheduler_state *state,
|
void handle_worker_blocked(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function is called when a worker that was blocked on an object that
|
* This function is called when a worker that was blocked on an object that
|
||||||
|
@ -223,9 +222,9 @@ void handle_worker_blocked(local_scheduler_state *state,
|
||||||
* @param worker The worker that is now unblocked.
|
* @param worker The worker that is now unblocked.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_worker_unblocked(local_scheduler_state *state,
|
void handle_worker_unblocked(LocalSchedulerState *state,
|
||||||
scheduling_algorithm_state *algorithm_state,
|
SchedulingAlgorithmState *algorithm_state,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This function fetches queued task's missing object dependencies. It is
|
* This function fetches queued task's missing object dependencies. It is
|
||||||
|
@ -248,7 +247,7 @@ int fetch_object_timeout_handler(event_loop *loop, timer_id id, void *context);
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void print_worker_info(const char *message,
|
void print_worker_info(const char *message,
|
||||||
scheduling_algorithm_state *algorithm_state);
|
SchedulingAlgorithmState *algorithm_state);
|
||||||
|
|
||||||
/** The following methods are for testing purposes only. */
|
/** The following methods are for testing purposes only. */
|
||||||
#ifdef PHOTON_TEST
|
#ifdef PHOTON_TEST
|
||||||
|
@ -259,7 +258,7 @@ void print_worker_info(const char *message,
|
||||||
* @param algorithm_state State maintained by the scheduling algorithm.
|
* @param algorithm_state State maintained by the scheduling algorithm.
|
||||||
* @return The number of tasks queued.
|
* @return The number of tasks queued.
|
||||||
*/
|
*/
|
||||||
int num_waiting_tasks(scheduling_algorithm_state *algorithm_state);
|
int num_waiting_tasks(SchedulingAlgorithmState *algorithm_state);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the number of tasks currently waiting for a worker to become available.
|
* Get the number of tasks currently waiting for a worker to become available.
|
||||||
|
@ -267,7 +266,7 @@ int num_waiting_tasks(scheduling_algorithm_state *algorithm_state);
|
||||||
* @param algorithm_state State maintained by the scheduling algorithm.
|
* @param algorithm_state State maintained by the scheduling algorithm.
|
||||||
* @return The number of tasks queued.
|
* @return The number of tasks queued.
|
||||||
*/
|
*/
|
||||||
int num_dispatch_tasks(scheduling_algorithm_state *algorithm_state);
|
int num_dispatch_tasks(SchedulingAlgorithmState *algorithm_state);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#endif /* PHOTON_ALGORITHM_H */
|
#endif /* PHOTON_ALGORITHM_H */
|
||||||
|
|
|
@ -4,8 +4,10 @@
|
||||||
#include "common/task.h"
|
#include "common/task.h"
|
||||||
#include <stdlib.h>
|
#include <stdlib.h>
|
||||||
|
|
||||||
photon_conn *photon_connect(const char *photon_socket, actor_id actor_id) {
|
PhotonConnection *PhotonConnection_init(const char *photon_socket,
|
||||||
photon_conn *result = malloc(sizeof(photon_conn));
|
ActorID actor_id) {
|
||||||
|
PhotonConnection *result =
|
||||||
|
(PhotonConnection *) malloc(sizeof(PhotonConnection));
|
||||||
result->conn = connect_ipc_sock_retry(photon_socket, -1, -1);
|
result->conn = connect_ipc_sock_retry(photon_socket, -1, -1);
|
||||||
register_worker_info info;
|
register_worker_info info;
|
||||||
memset(&info, 0, sizeof(info));
|
memset(&info, 0, sizeof(info));
|
||||||
|
@ -18,19 +20,19 @@ photon_conn *photon_connect(const char *photon_socket, actor_id actor_id) {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_disconnect(photon_conn *conn) {
|
void PhotonConnection_free(PhotonConnection *conn) {
|
||||||
close(conn->conn);
|
close(conn->conn);
|
||||||
free(conn);
|
free(conn);
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_log_event(photon_conn *conn,
|
void photon_log_event(PhotonConnection *conn,
|
||||||
uint8_t *key,
|
uint8_t *key,
|
||||||
int64_t key_length,
|
int64_t key_length,
|
||||||
uint8_t *value,
|
uint8_t *value,
|
||||||
int64_t value_length) {
|
int64_t value_length) {
|
||||||
int64_t message_length =
|
int64_t message_length =
|
||||||
sizeof(key_length) + sizeof(value_length) + key_length + value_length;
|
sizeof(key_length) + sizeof(value_length) + key_length + value_length;
|
||||||
uint8_t *message = malloc(message_length);
|
uint8_t *message = (uint8_t *) malloc(message_length);
|
||||||
int64_t offset = 0;
|
int64_t offset = 0;
|
||||||
memcpy(&message[offset], &key_length, sizeof(key_length));
|
memcpy(&message[offset], &key_length, sizeof(key_length));
|
||||||
offset += sizeof(key_length);
|
offset += sizeof(key_length);
|
||||||
|
@ -45,12 +47,12 @@ void photon_log_event(photon_conn *conn,
|
||||||
free(message);
|
free(message);
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_submit(photon_conn *conn, task_spec *task) {
|
void photon_submit(PhotonConnection *conn, task_spec *task) {
|
||||||
write_message(conn->conn, SUBMIT_TASK, task_spec_size(task),
|
write_message(conn->conn, SUBMIT_TASK, task_spec_size(task),
|
||||||
(uint8_t *) task);
|
(uint8_t *) task);
|
||||||
}
|
}
|
||||||
|
|
||||||
task_spec *photon_get_task(photon_conn *conn) {
|
task_spec *photon_get_task(PhotonConnection *conn) {
|
||||||
write_message(conn->conn, GET_TASK, 0, NULL);
|
write_message(conn->conn, GET_TASK, 0, NULL);
|
||||||
int64_t type;
|
int64_t type;
|
||||||
int64_t length;
|
int64_t length;
|
||||||
|
@ -64,19 +66,19 @@ task_spec *photon_get_task(photon_conn *conn) {
|
||||||
return task;
|
return task;
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_task_done(photon_conn *conn) {
|
void photon_task_done(PhotonConnection *conn) {
|
||||||
write_message(conn->conn, TASK_DONE, 0, NULL);
|
write_message(conn->conn, TASK_DONE, 0, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_reconstruct_object(photon_conn *conn, object_id object_id) {
|
void photon_reconstruct_object(PhotonConnection *conn, ObjectID object_id) {
|
||||||
write_message(conn->conn, RECONSTRUCT_OBJECT, sizeof(object_id),
|
write_message(conn->conn, RECONSTRUCT_OBJECT, sizeof(object_id),
|
||||||
(uint8_t *) &object_id);
|
(uint8_t *) &object_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_log_message(photon_conn *conn) {
|
void photon_log_message(PhotonConnection *conn) {
|
||||||
write_message(conn->conn, LOG_MESSAGE, 0, NULL);
|
write_message(conn->conn, LOG_MESSAGE, 0, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
void photon_notify_unblocked(photon_conn *conn) {
|
void photon_notify_unblocked(PhotonConnection *conn) {
|
||||||
write_message(conn->conn, NOTIFY_UNBLOCKED, 0, NULL);
|
write_message(conn->conn, NOTIFY_UNBLOCKED, 0, NULL);
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,7 +7,7 @@
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/* File descriptor of the Unix domain socket that connects to photon. */
|
/* File descriptor of the Unix domain socket that connects to photon. */
|
||||||
int conn;
|
int conn;
|
||||||
} photon_conn;
|
} PhotonConnection;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Connect to the local scheduler.
|
* Connect to the local scheduler.
|
||||||
|
@ -18,15 +18,16 @@ typedef struct {
|
||||||
* running on this actor, this should be NIL_ACTOR_ID.
|
* running on this actor, this should be NIL_ACTOR_ID.
|
||||||
* @return The connection information.
|
* @return The connection information.
|
||||||
*/
|
*/
|
||||||
photon_conn *photon_connect(const char *photon_socket, actor_id actor_id);
|
PhotonConnection *PhotonConnection_init(const char *photon_socket,
|
||||||
|
ActorID actor_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Disconnect from the local scheduler.
|
* Disconnect from the local scheduler.
|
||||||
*
|
*
|
||||||
* @param conn Photon connection information returned by photon_connect.
|
* @param conn Photon connection information returned by PhotonConnection_init.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_disconnect(photon_conn *conn);
|
void PhotonConnection_free(PhotonConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submit a task to the local scheduler.
|
* Submit a task to the local scheduler.
|
||||||
|
@ -35,7 +36,7 @@ void photon_disconnect(photon_conn *conn);
|
||||||
* @param task The address of the task to submit.
|
* @param task The address of the task to submit.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_submit(photon_conn *conn, task_spec *task);
|
void photon_submit(PhotonConnection *conn, task_spec *task);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Log an event to the event log. This will call RPUSH key value. We use RPUSH
|
* Log an event to the event log. This will call RPUSH key value. We use RPUSH
|
||||||
|
@ -50,7 +51,7 @@ void photon_submit(photon_conn *conn, task_spec *task);
|
||||||
* @param value_length The length of the value.
|
* @param value_length The length of the value.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_log_event(photon_conn *conn,
|
void photon_log_event(PhotonConnection *conn,
|
||||||
uint8_t *key,
|
uint8_t *key,
|
||||||
int64_t key_length,
|
int64_t key_length,
|
||||||
uint8_t *value,
|
uint8_t *value,
|
||||||
|
@ -66,7 +67,7 @@ void photon_log_event(photon_conn *conn,
|
||||||
* @param conn The connection information.
|
* @param conn The connection information.
|
||||||
* @return The address of the assigned task.
|
* @return The address of the assigned task.
|
||||||
*/
|
*/
|
||||||
task_spec *photon_get_task(photon_conn *conn);
|
task_spec *photon_get_task(PhotonConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tell the local scheduler that the client has finished executing a task.
|
* Tell the local scheduler that the client has finished executing a task.
|
||||||
|
@ -74,7 +75,7 @@ task_spec *photon_get_task(photon_conn *conn);
|
||||||
* @param conn The connection information.
|
* @param conn The connection information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_task_done(photon_conn *conn);
|
void photon_task_done(PhotonConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tell the local scheduler to reconstruct an object.
|
* Tell the local scheduler to reconstruct an object.
|
||||||
|
@ -83,7 +84,7 @@ void photon_task_done(photon_conn *conn);
|
||||||
* @param object_id The ID of the object to reconstruct.
|
* @param object_id The ID of the object to reconstruct.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_reconstruct_object(photon_conn *conn, object_id object_id);
|
void photon_reconstruct_object(PhotonConnection *conn, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Send a log message to the local scheduler.
|
* Send a log message to the local scheduler.
|
||||||
|
@ -91,7 +92,7 @@ void photon_reconstruct_object(photon_conn *conn, object_id object_id);
|
||||||
* @param conn The connection information.
|
* @param conn The connection information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_log_message(photon_conn *conn);
|
void photon_log_message(PhotonConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Notify the local scheduler that this client (worker) is no longer blocked.
|
* Notify the local scheduler that this client (worker) is no longer blocked.
|
||||||
|
@ -99,6 +100,6 @@ void photon_log_message(photon_conn *conn);
|
||||||
* @param conn The connection information.
|
* @param conn The connection information.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void photon_notify_unblocked(photon_conn *conn);
|
void photon_notify_unblocked(PhotonConnection *conn);
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
|
@ -9,7 +9,7 @@ PyObject *PhotonError;
|
||||||
// clang-format off
|
// clang-format off
|
||||||
typedef struct {
|
typedef struct {
|
||||||
PyObject_HEAD
|
PyObject_HEAD
|
||||||
photon_conn *photon_connection;
|
PhotonConnection *photon_connection;
|
||||||
} PyPhotonClient;
|
} PyPhotonClient;
|
||||||
// clang-format on
|
// clang-format on
|
||||||
|
|
||||||
|
@ -17,18 +17,18 @@ static int PyPhotonClient_init(PyPhotonClient *self,
|
||||||
PyObject *args,
|
PyObject *args,
|
||||||
PyObject *kwds) {
|
PyObject *kwds) {
|
||||||
char *socket_name;
|
char *socket_name;
|
||||||
actor_id actor_id;
|
ActorID actor_id;
|
||||||
if (!PyArg_ParseTuple(args, "sO&", &socket_name, PyStringToUniqueID,
|
if (!PyArg_ParseTuple(args, "sO&", &socket_name, PyStringToUniqueID,
|
||||||
&actor_id)) {
|
&actor_id)) {
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
/* Connect to the Photon scheduler. */
|
/* Connect to the Photon scheduler. */
|
||||||
self->photon_connection = photon_connect(socket_name, actor_id);
|
self->photon_connection = PhotonConnection_init(socket_name, actor_id);
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void PyPhotonClient_dealloc(PyPhotonClient *self) {
|
static void PyPhotonClient_dealloc(PyPhotonClient *self) {
|
||||||
photon_disconnect(((PyPhotonClient *) self)->photon_connection);
|
PhotonConnection_free(((PyPhotonClient *) self)->photon_connection);
|
||||||
Py_TYPE(self)->tp_free((PyObject *) self);
|
Py_TYPE(self)->tp_free((PyObject *) self);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -56,7 +56,7 @@ static PyObject *PyPhotonClient_get_task(PyObject *self) {
|
||||||
|
|
||||||
static PyObject *PyPhotonClient_reconstruct_object(PyObject *self,
|
static PyObject *PyPhotonClient_reconstruct_object(PyObject *self,
|
||||||
PyObject *args) {
|
PyObject *args) {
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&", PyStringToUniqueID, &object_id)) {
|
if (!PyArg_ParseTuple(args, "O&", PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,8 +22,8 @@
|
||||||
#include "utarray.h"
|
#include "utarray.h"
|
||||||
#include "uthash.h"
|
#include "uthash.h"
|
||||||
|
|
||||||
UT_icd task_ptr_icd = {sizeof(task *), NULL, NULL, NULL};
|
UT_icd task_ptr_icd = {sizeof(Task *), NULL, NULL, NULL};
|
||||||
UT_icd workers_icd = {sizeof(local_scheduler_client *), NULL, NULL, NULL};
|
UT_icd workers_icd = {sizeof(LocalSchedulerClient *), NULL, NULL, NULL};
|
||||||
|
|
||||||
UT_icd pid_t_icd = {sizeof(pid_t), NULL, NULL, NULL};
|
UT_icd pid_t_icd = {sizeof(pid_t), NULL, NULL, NULL};
|
||||||
|
|
||||||
|
@ -36,7 +36,7 @@ UT_icd byte_icd = {sizeof(uint8_t), NULL, NULL, NULL};
|
||||||
* @param spec Task specification object.
|
* @param spec Task specification object.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void print_resource_info(const local_scheduler_state *state,
|
void print_resource_info(const LocalSchedulerState *state,
|
||||||
const task_spec *spec) {
|
const task_spec *spec) {
|
||||||
#if RAY_COMMON_LOG_LEVEL <= RAY_COMMON_DEBUG
|
#if RAY_COMMON_LOG_LEVEL <= RAY_COMMON_DEBUG
|
||||||
/* Print information about available and requested resources. */
|
/* Print information about available and requested resources. */
|
||||||
|
@ -58,7 +58,7 @@ void print_resource_info(const local_scheduler_state *state,
|
||||||
}
|
}
|
||||||
|
|
||||||
int force_kill_worker(event_loop *loop, timer_id id, void *context) {
|
int force_kill_worker(event_loop *loop, timer_id id, void *context) {
|
||||||
local_scheduler_client *worker = (local_scheduler_client *) context;
|
LocalSchedulerClient *worker = (LocalSchedulerClient *) context;
|
||||||
kill(worker->pid, SIGKILL);
|
kill(worker->pid, SIGKILL);
|
||||||
close(worker->sock);
|
close(worker->sock);
|
||||||
free(worker);
|
free(worker);
|
||||||
|
@ -76,13 +76,13 @@ int force_kill_worker(event_loop *loop, timer_id id, void *context) {
|
||||||
* to clean up its own state.
|
* to clean up its own state.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void kill_worker(local_scheduler_client *worker, bool cleanup) {
|
void kill_worker(LocalSchedulerClient *worker, bool cleanup) {
|
||||||
/* Erase the local scheduler's reference to the worker. */
|
/* Erase the local scheduler's reference to the worker. */
|
||||||
local_scheduler_state *state = worker->local_scheduler_state;
|
LocalSchedulerState *state = worker->local_scheduler_state;
|
||||||
int num_workers = utarray_len(state->workers);
|
int num_workers = utarray_len(state->workers);
|
||||||
for (int i = 0; i < utarray_len(state->workers); ++i) {
|
for (int i = 0; i < utarray_len(state->workers); ++i) {
|
||||||
local_scheduler_client *active_worker =
|
LocalSchedulerClient *active_worker =
|
||||||
*(local_scheduler_client **) utarray_eltptr(state->workers, i);
|
*(LocalSchedulerClient **) utarray_eltptr(state->workers, i);
|
||||||
if (active_worker == worker) {
|
if (active_worker == worker) {
|
||||||
utarray_erase(state->workers, i, 1);
|
utarray_erase(state->workers, i, 1);
|
||||||
}
|
}
|
||||||
|
@ -124,14 +124,14 @@ void kill_worker(local_scheduler_client *worker, bool cleanup) {
|
||||||
/* Clean up the task in progress. */
|
/* Clean up the task in progress. */
|
||||||
if (worker->task_in_progress) {
|
if (worker->task_in_progress) {
|
||||||
/* Return the resources that the worker was using. */
|
/* Return the resources that the worker was using. */
|
||||||
task_spec *spec = task_task_spec(worker->task_in_progress);
|
task_spec *spec = Task_task_spec(worker->task_in_progress);
|
||||||
update_dynamic_resources(state, spec, true);
|
update_dynamic_resources(state, spec, true);
|
||||||
/* Update the task table to reflect that the task failed to complete. */
|
/* Update the task table to reflect that the task failed to complete. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
task_set_state(worker->task_in_progress, TASK_STATUS_LOST);
|
Task_set_state(worker->task_in_progress, TASK_STATUS_LOST);
|
||||||
task_table_update(state->db, worker->task_in_progress, NULL, NULL, NULL);
|
task_table_update(state->db, worker->task_in_progress, NULL, NULL, NULL);
|
||||||
} else {
|
} else {
|
||||||
free_task(worker->task_in_progress);
|
Task_free(worker->task_in_progress);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -144,7 +144,7 @@ void kill_worker(local_scheduler_client *worker, bool cleanup) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_local_scheduler(local_scheduler_state *state) {
|
void LocalSchedulerState_free(LocalSchedulerState *state) {
|
||||||
/* Free the command for starting new workers. */
|
/* Free the command for starting new workers. */
|
||||||
if (state->config.start_worker_command != NULL) {
|
if (state->config.start_worker_command != NULL) {
|
||||||
int i = 0;
|
int i = 0;
|
||||||
|
@ -173,10 +173,10 @@ void free_local_scheduler(local_scheduler_state *state) {
|
||||||
* workers. */
|
* workers. */
|
||||||
/* TODO(swang): It's possible that the local scheduler will exit before all
|
/* TODO(swang): It's possible that the local scheduler will exit before all
|
||||||
* of its task table updates make it to redis. */
|
* of its task table updates make it to redis. */
|
||||||
for (local_scheduler_client **worker =
|
for (LocalSchedulerClient **worker =
|
||||||
(local_scheduler_client **) utarray_front(state->workers);
|
(LocalSchedulerClient **) utarray_front(state->workers);
|
||||||
worker != NULL;
|
worker != NULL;
|
||||||
worker = (local_scheduler_client **) utarray_front(state->workers)) {
|
worker = (LocalSchedulerClient **) utarray_front(state->workers)) {
|
||||||
kill_worker(*worker, true);
|
kill_worker(*worker, true);
|
||||||
}
|
}
|
||||||
utarray_free(state->workers);
|
utarray_free(state->workers);
|
||||||
|
@ -201,7 +201,7 @@ void free_local_scheduler(local_scheduler_state *state) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Free the algorithm state. */
|
/* Free the algorithm state. */
|
||||||
free_scheduling_algorithm_state(state->algorithm_state);
|
SchedulingAlgorithmState_free(state->algorithm_state);
|
||||||
state->algorithm_state = NULL;
|
state->algorithm_state = NULL;
|
||||||
/* Free the input buffer. */
|
/* Free the input buffer. */
|
||||||
utarray_free(state->input_buffer);
|
utarray_free(state->input_buffer);
|
||||||
|
@ -219,7 +219,7 @@ void free_local_scheduler(local_scheduler_state *state) {
|
||||||
* @param state The state of the local scheduler.
|
* @param state The state of the local scheduler.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void start_worker(local_scheduler_state *state, actor_id actor_id) {
|
void start_worker(LocalSchedulerState *state, ActorID actor_id) {
|
||||||
/* We can't start a worker if we don't have the path to the worker script. */
|
/* We can't start a worker if we don't have the path to the worker script. */
|
||||||
if (state->config.start_worker_command == NULL) {
|
if (state->config.start_worker_command == NULL) {
|
||||||
LOG_WARN("No valid command to start worker provided. Cannot start worker.");
|
LOG_WARN("No valid command to start worker provided. Cannot start worker.");
|
||||||
|
@ -234,7 +234,7 @@ void start_worker(local_scheduler_state *state, actor_id actor_id) {
|
||||||
}
|
}
|
||||||
|
|
||||||
char id_string[ID_STRING_SIZE];
|
char id_string[ID_STRING_SIZE];
|
||||||
object_id_to_string(actor_id, id_string, ID_STRING_SIZE);
|
ObjectID_to_string(actor_id, id_string, ID_STRING_SIZE);
|
||||||
/* Figure out how many arguments there are in the start_worker_command. */
|
/* Figure out how many arguments there are in the start_worker_command. */
|
||||||
int num_args = 0;
|
int num_args = 0;
|
||||||
for (; state->config.start_worker_command[num_args] != NULL; ++num_args) {
|
for (; state->config.start_worker_command[num_args] != NULL; ++num_args) {
|
||||||
|
@ -251,7 +251,7 @@ void start_worker(local_scheduler_state *state, actor_id actor_id) {
|
||||||
execvp(start_actor_worker_command[0],
|
execvp(start_actor_worker_command[0],
|
||||||
(char *const *) start_actor_worker_command);
|
(char *const *) start_actor_worker_command);
|
||||||
free(start_actor_worker_command);
|
free(start_actor_worker_command);
|
||||||
free_local_scheduler(state);
|
LocalSchedulerState_free(state);
|
||||||
LOG_FATAL("Failed to start worker");
|
LOG_FATAL("Failed to start worker");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -296,7 +296,7 @@ const char **parse_command(const char *command) {
|
||||||
return command_args;
|
return command_args;
|
||||||
}
|
}
|
||||||
|
|
||||||
local_scheduler_state *init_local_scheduler(
|
LocalSchedulerState *LocalSchedulerState_init(
|
||||||
const char *node_ip_address,
|
const char *node_ip_address,
|
||||||
event_loop *loop,
|
event_loop *loop,
|
||||||
const char *redis_addr,
|
const char *redis_addr,
|
||||||
|
@ -309,7 +309,7 @@ local_scheduler_state *init_local_scheduler(
|
||||||
const double static_resource_conf[],
|
const double static_resource_conf[],
|
||||||
const char *start_worker_command,
|
const char *start_worker_command,
|
||||||
int num_workers) {
|
int num_workers) {
|
||||||
local_scheduler_state *state = malloc(sizeof(local_scheduler_state));
|
LocalSchedulerState *state = malloc(sizeof(LocalSchedulerState));
|
||||||
/* Set the configuration struct for the local scheduler. */
|
/* Set the configuration struct for the local scheduler. */
|
||||||
if (start_worker_command != NULL) {
|
if (start_worker_command != NULL) {
|
||||||
state->config.start_worker_command = parse_command(start_worker_command);
|
state->config.start_worker_command = parse_command(start_worker_command);
|
||||||
|
@ -380,7 +380,7 @@ local_scheduler_state *init_local_scheduler(
|
||||||
event_loop_add_file(loop, plasma_fd, EVENT_LOOP_READ,
|
event_loop_add_file(loop, plasma_fd, EVENT_LOOP_READ,
|
||||||
process_plasma_notification, state);
|
process_plasma_notification, state);
|
||||||
/* Add scheduler state. */
|
/* Add scheduler state. */
|
||||||
state->algorithm_state = make_scheduling_algorithm_state();
|
state->algorithm_state = SchedulingAlgorithmState_init();
|
||||||
/* Add the input buffer. This is used to read in messages from clients without
|
/* Add the input buffer. This is used to read in messages from clients without
|
||||||
* having to reallocate a new buffer every time. */
|
* having to reallocate a new buffer every time. */
|
||||||
utarray_new(state->input_buffer, &byte_icd);
|
utarray_new(state->input_buffer, &byte_icd);
|
||||||
|
@ -402,7 +402,7 @@ local_scheduler_state *init_local_scheduler(
|
||||||
return state;
|
return state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void update_dynamic_resources(local_scheduler_state *state,
|
void update_dynamic_resources(LocalSchedulerState *state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool return_resources) {
|
bool return_resources) {
|
||||||
for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) {
|
for (int i = 0; i < MAX_RESOURCE_INDEX; ++i) {
|
||||||
|
@ -425,9 +425,9 @@ void update_dynamic_resources(local_scheduler_state *state,
|
||||||
print_resource_info(state, spec);
|
print_resource_info(state, spec);
|
||||||
}
|
}
|
||||||
|
|
||||||
void assign_task_to_worker(local_scheduler_state *state,
|
void assign_task_to_worker(LocalSchedulerState *state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
local_scheduler_client *worker) {
|
LocalSchedulerClient *worker) {
|
||||||
if (write_message(worker->sock, EXECUTE_TASK, task_spec_size(spec),
|
if (write_message(worker->sock, EXECUTE_TASK, task_spec_size(spec),
|
||||||
(uint8_t *) spec) < 0) {
|
(uint8_t *) spec) < 0) {
|
||||||
if (errno == EPIPE || errno == EBADF) {
|
if (errno == EPIPE || errno == EBADF) {
|
||||||
|
@ -445,17 +445,17 @@ void assign_task_to_worker(local_scheduler_state *state,
|
||||||
/* Resource accounting:
|
/* Resource accounting:
|
||||||
* Update dynamic resource vector in the local scheduler state. */
|
* Update dynamic resource vector in the local scheduler state. */
|
||||||
update_dynamic_resources(state, spec, false);
|
update_dynamic_resources(state, spec, false);
|
||||||
task *task = alloc_task(spec, TASK_STATUS_RUNNING,
|
Task *task = Task_alloc(spec, TASK_STATUS_RUNNING,
|
||||||
state->db ? get_db_client_id(state->db) : NIL_ID);
|
state->db ? get_db_client_id(state->db) : NIL_ID);
|
||||||
/* Record which task this worker is executing. This will be freed in
|
/* 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
|
* process_message when the worker sends a GET_TASK message to the local
|
||||||
* scheduler. */
|
* scheduler. */
|
||||||
worker->task_in_progress = copy_task(task);
|
worker->task_in_progress = Task_copy(task);
|
||||||
/* Update the global task table. */
|
/* Update the global task table. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
task_table_update(state->db, task, NULL, NULL, NULL);
|
task_table_update(state->db, task, NULL, NULL, NULL);
|
||||||
} else {
|
} else {
|
||||||
free_task(task);
|
Task_free(task);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -463,9 +463,9 @@ void process_plasma_notification(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
local_scheduler_state *state = context;
|
LocalSchedulerState *state = context;
|
||||||
/* Read the notification from Plasma. */
|
/* Read the notification from Plasma. */
|
||||||
object_info object_info;
|
ObjectInfo object_info;
|
||||||
int error =
|
int error =
|
||||||
read_bytes(client_sock, (uint8_t *) &object_info, sizeof(object_info));
|
read_bytes(client_sock, (uint8_t *) &object_info, sizeof(object_info));
|
||||||
if (error < 0) {
|
if (error < 0) {
|
||||||
|
@ -485,7 +485,7 @@ void process_plasma_notification(event_loop *loop,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void reconstruct_task_update_callback(task *task, void *user_context) {
|
void reconstruct_task_update_callback(Task *task, void *user_context) {
|
||||||
if (task == NULL) {
|
if (task == NULL) {
|
||||||
/* The test-and-set of the task's scheduling state failed, so the task was
|
/* The test-and-set of the task's scheduling state failed, so the task was
|
||||||
* either not finished yet, or it was already being reconstructed.
|
* either not finished yet, or it was already being reconstructed.
|
||||||
|
@ -494,30 +494,30 @@ void reconstruct_task_update_callback(task *task, void *user_context) {
|
||||||
}
|
}
|
||||||
/* Otherwise, the test-and-set succeeded, so resubmit the task for execution
|
/* Otherwise, the test-and-set succeeded, so resubmit the task for execution
|
||||||
* to ensure that reconstruction will happen. */
|
* to ensure that reconstruction will happen. */
|
||||||
local_scheduler_state *state = user_context;
|
LocalSchedulerState *state = user_context;
|
||||||
task_spec *spec = task_task_spec(task);
|
task_spec *spec = Task_task_spec(task);
|
||||||
/* If the task is an actor task, then we currently do not reconstruct it.
|
/* If the task is an actor task, then we currently do not reconstruct it.
|
||||||
* TODO(rkn): Handle this better. */
|
* TODO(rkn): Handle this better. */
|
||||||
CHECK(actor_ids_equal(task_spec_actor_id(spec), NIL_ACTOR_ID));
|
CHECK(ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID));
|
||||||
/* Resubmit the task. */
|
/* Resubmit the task. */
|
||||||
handle_task_submitted(state, state->algorithm_state, spec);
|
handle_task_submitted(state, state->algorithm_state, spec);
|
||||||
/* Recursively reconstruct the task's inputs, if necessary. */
|
/* Recursively reconstruct the task's inputs, if necessary. */
|
||||||
for (int64_t i = 0; i < task_num_args(spec); ++i) {
|
for (int64_t i = 0; i < task_num_args(spec); ++i) {
|
||||||
if (task_arg_type(spec, i) == ARG_BY_REF) {
|
if (task_arg_type(spec, i) == ARG_BY_REF) {
|
||||||
object_id arg_id = task_arg_id(spec, i);
|
ObjectID arg_id = task_arg_id(spec, i);
|
||||||
reconstruct_object(state, arg_id);
|
reconstruct_object(state, arg_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void reconstruct_evicted_result_lookup_callback(object_id reconstruct_object_id,
|
void reconstruct_evicted_result_lookup_callback(ObjectID reconstruct_object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
/* TODO(swang): The following check will fail if an object was created by a
|
/* TODO(swang): The following check will fail if an object was created by a
|
||||||
* put. */
|
* put. */
|
||||||
CHECKM(!IS_NIL_ID(task_id),
|
CHECKM(!IS_NIL_ID(task_id),
|
||||||
"No task information found for object during reconstruction");
|
"No task information found for object during reconstruction");
|
||||||
local_scheduler_state *state = user_context;
|
LocalSchedulerState *state = user_context;
|
||||||
/* If there are no other instances of the task running, it's safe for us to
|
/* If there are no other instances of the task running, it's safe for us to
|
||||||
* claim responsibility for reconstruction. */
|
* claim responsibility for reconstruction. */
|
||||||
task_table_test_and_update(state->db, task_id,
|
task_table_test_and_update(state->db, task_id,
|
||||||
|
@ -526,8 +526,8 @@ void reconstruct_evicted_result_lookup_callback(object_id reconstruct_object_id,
|
||||||
reconstruct_task_update_callback, state);
|
reconstruct_task_update_callback, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void reconstruct_failed_result_lookup_callback(object_id reconstruct_object_id,
|
void reconstruct_failed_result_lookup_callback(ObjectID reconstruct_object_id,
|
||||||
task_id task_id,
|
TaskID task_id,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
/* TODO(swang): The following check will fail if an object was created by a
|
/* TODO(swang): The following check will fail if an object was created by a
|
||||||
* put. */
|
* put. */
|
||||||
|
@ -541,7 +541,7 @@ void reconstruct_failed_result_lookup_callback(object_id reconstruct_object_id,
|
||||||
"entry yet)");
|
"entry yet)");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
local_scheduler_state *state = user_context;
|
LocalSchedulerState *state = user_context;
|
||||||
/* If the task failed to finish, it's safe for us to claim responsibility for
|
/* If the task failed to finish, it's safe for us to claim responsibility for
|
||||||
* reconstruction. */
|
* reconstruction. */
|
||||||
task_table_test_and_update(state->db, task_id, TASK_STATUS_LOST,
|
task_table_test_and_update(state->db, task_id, TASK_STATUS_LOST,
|
||||||
|
@ -549,7 +549,7 @@ void reconstruct_failed_result_lookup_callback(object_id reconstruct_object_id,
|
||||||
reconstruct_task_update_callback, state);
|
reconstruct_task_update_callback, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void reconstruct_object_lookup_callback(object_id reconstruct_object_id,
|
void reconstruct_object_lookup_callback(ObjectID reconstruct_object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
|
@ -557,7 +557,7 @@ void reconstruct_object_lookup_callback(object_id reconstruct_object_id,
|
||||||
/* Only continue reconstruction if we find that the object doesn't exist on
|
/* Only continue reconstruction if we find that the object doesn't exist on
|
||||||
* any nodes. NOTE: This codepath is not responsible for checking if the
|
* any nodes. NOTE: This codepath is not responsible for checking if the
|
||||||
* object table entry is up-to-date. */
|
* object table entry is up-to-date. */
|
||||||
local_scheduler_state *state = user_context;
|
LocalSchedulerState *state = user_context;
|
||||||
/* Look up the task that created the object in the result table. */
|
/* Look up the task that created the object in the result table. */
|
||||||
if (manager_count == 0) {
|
if (manager_count == 0) {
|
||||||
/* If the object was created and later evicted, we reconstruct the object
|
/* If the object was created and later evicted, we reconstruct the object
|
||||||
|
@ -574,8 +574,8 @@ void reconstruct_object_lookup_callback(object_id reconstruct_object_id,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void reconstruct_object(local_scheduler_state *state,
|
void reconstruct_object(LocalSchedulerState *state,
|
||||||
object_id reconstruct_object_id) {
|
ObjectID reconstruct_object_id) {
|
||||||
LOG_DEBUG("Starting reconstruction");
|
LOG_DEBUG("Starting reconstruction");
|
||||||
/* TODO(swang): Track task lineage for puts. */
|
/* TODO(swang): Track task lineage for puts. */
|
||||||
CHECK(state->db != NULL);
|
CHECK(state->db != NULL);
|
||||||
|
@ -589,8 +589,8 @@ void process_message(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
local_scheduler_client *worker = context;
|
LocalSchedulerClient *worker = context;
|
||||||
local_scheduler_state *state = worker->local_scheduler_state;
|
LocalSchedulerState *state = worker->local_scheduler_state;
|
||||||
|
|
||||||
int64_t type;
|
int64_t type;
|
||||||
int64_t length = read_buffer(client_sock, &type, state->input_buffer);
|
int64_t length = read_buffer(client_sock, &type, state->input_buffer);
|
||||||
|
@ -603,15 +603,15 @@ void process_message(event_loop *loop,
|
||||||
/* Update the result table, which holds mappings of object ID -> ID of the
|
/* Update the result table, which holds mappings of object ID -> ID of the
|
||||||
* task that created it. */
|
* task that created it. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
task_id task_id = task_spec_id(spec);
|
TaskID task_id = task_spec_id(spec);
|
||||||
for (int64_t i = 0; i < task_num_returns(spec); ++i) {
|
for (int64_t i = 0; i < task_num_returns(spec); ++i) {
|
||||||
object_id return_id = task_return(spec, i);
|
ObjectID return_id = task_return(spec, i);
|
||||||
result_table_add(state->db, return_id, task_id, NULL, NULL, NULL);
|
result_table_add(state->db, return_id, task_id, NULL, NULL, NULL);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Handle the task submission. */
|
/* Handle the task submission. */
|
||||||
if (actor_ids_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) {
|
if (ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) {
|
||||||
handle_task_submitted(state, state->algorithm_state, spec);
|
handle_task_submitted(state, state->algorithm_state, spec);
|
||||||
} else {
|
} else {
|
||||||
handle_actor_task_submitted(state, state->algorithm_state, spec);
|
handle_actor_task_submitted(state, state->algorithm_state, spec);
|
||||||
|
@ -639,7 +639,7 @@ void process_message(event_loop *loop,
|
||||||
offset += value_length;
|
offset += value_length;
|
||||||
CHECK(offset == length);
|
CHECK(offset == length);
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
ray_log_event(state->db, key, key_length, value, value_length);
|
RayLogger_log_event(state->db, key, key_length, value, value_length);
|
||||||
}
|
}
|
||||||
free(key);
|
free(key);
|
||||||
free(value);
|
free(value);
|
||||||
|
@ -649,17 +649,17 @@ void process_message(event_loop *loop,
|
||||||
* running on the worker). */
|
* running on the worker). */
|
||||||
register_worker_info *info =
|
register_worker_info *info =
|
||||||
(register_worker_info *) utarray_front(state->input_buffer);
|
(register_worker_info *) utarray_front(state->input_buffer);
|
||||||
if (!actor_ids_equal(info->actor_id, NIL_ACTOR_ID)) {
|
if (!ActorID_equal(info->actor_id, NIL_ACTOR_ID)) {
|
||||||
/* Make sure that the local scheduler is aware that it is responsible for
|
/* Make sure that the local scheduler is aware that it is responsible for
|
||||||
* this actor. */
|
* this actor. */
|
||||||
actor_map_entry *entry;
|
actor_map_entry *entry;
|
||||||
HASH_FIND(hh, state->actor_mapping, &info->actor_id,
|
HASH_FIND(hh, state->actor_mapping, &info->actor_id,
|
||||||
sizeof(info->actor_id), entry);
|
sizeof(info->actor_id), entry);
|
||||||
CHECK(entry != NULL);
|
CHECK(entry != NULL);
|
||||||
CHECK(db_client_ids_equal(entry->local_scheduler_id,
|
CHECK(DBClientID_equal(entry->local_scheduler_id,
|
||||||
get_db_client_id(state->db)));
|
get_db_client_id(state->db)));
|
||||||
/* Update the worker struct with this actor ID. */
|
/* Update the worker struct with this actor ID. */
|
||||||
CHECK(actor_ids_equal(worker->actor_id, NIL_ACTOR_ID));
|
CHECK(ActorID_equal(worker->actor_id, NIL_ACTOR_ID));
|
||||||
worker->actor_id = info->actor_id;
|
worker->actor_id = info->actor_id;
|
||||||
/* Let the scheduling algorithm process the presence of this new
|
/* Let the scheduling algorithm process the presence of this new
|
||||||
* worker. */
|
* worker. */
|
||||||
|
@ -691,25 +691,25 @@ void process_message(event_loop *loop,
|
||||||
case GET_TASK: {
|
case GET_TASK: {
|
||||||
/* If this worker reports a completed task: account for resources. */
|
/* If this worker reports a completed task: account for resources. */
|
||||||
if (worker->task_in_progress != NULL) {
|
if (worker->task_in_progress != NULL) {
|
||||||
task_spec *spec = task_task_spec(worker->task_in_progress);
|
task_spec *spec = Task_task_spec(worker->task_in_progress);
|
||||||
/* Return dynamic resources back for the task in progress. */
|
/* Return dynamic resources back for the task in progress. */
|
||||||
update_dynamic_resources(state, spec, true);
|
update_dynamic_resources(state, spec, true);
|
||||||
/* If we're connected to Redis, update tables. */
|
/* If we're connected to Redis, update tables. */
|
||||||
if (state->db != NULL) {
|
if (state->db != NULL) {
|
||||||
/* Update control state tables. */
|
/* Update control state tables. */
|
||||||
task_set_state(worker->task_in_progress, TASK_STATUS_DONE);
|
Task_set_state(worker->task_in_progress, TASK_STATUS_DONE);
|
||||||
task_table_update(state->db, worker->task_in_progress, NULL, NULL,
|
task_table_update(state->db, worker->task_in_progress, NULL, NULL,
|
||||||
NULL);
|
NULL);
|
||||||
/* The call to task_table_update takes ownership of the
|
/* The call to task_table_update takes ownership of the
|
||||||
* task_in_progress, so we set the pointer to NULL so it is not used. */
|
* task_in_progress, so we set the pointer to NULL so it is not used. */
|
||||||
} else {
|
} else {
|
||||||
free_task(worker->task_in_progress);
|
Task_free(worker->task_in_progress);
|
||||||
}
|
}
|
||||||
worker->task_in_progress = NULL;
|
worker->task_in_progress = NULL;
|
||||||
}
|
}
|
||||||
/* Let the scheduling algorithm process the fact that there is an available
|
/* Let the scheduling algorithm process the fact that there is an available
|
||||||
* worker. */
|
* worker. */
|
||||||
if (actor_ids_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
if (ActorID_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
||||||
handle_worker_available(state, state->algorithm_state, worker);
|
handle_worker_available(state, state->algorithm_state, worker);
|
||||||
} else {
|
} else {
|
||||||
handle_actor_worker_available(state, state->algorithm_state, worker);
|
handle_actor_worker_available(state, state->algorithm_state, worker);
|
||||||
|
@ -718,7 +718,7 @@ void process_message(event_loop *loop,
|
||||||
case RECONSTRUCT_OBJECT: {
|
case RECONSTRUCT_OBJECT: {
|
||||||
if (worker->task_in_progress != NULL && !worker->is_blocked) {
|
if (worker->task_in_progress != NULL && !worker->is_blocked) {
|
||||||
/* TODO(swang): For now, we don't handle blocked actors. */
|
/* TODO(swang): For now, we don't handle blocked actors. */
|
||||||
if (actor_ids_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
if (ActorID_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
||||||
/* If the worker was executing a task (i.e. non-driver) and it wasn't
|
/* If the worker was executing a task (i.e. non-driver) and it wasn't
|
||||||
* already blocked on an object that's not locally available, update its
|
* already blocked on an object that's not locally available, update its
|
||||||
* state to blocked. */
|
* state to blocked. */
|
||||||
|
@ -726,13 +726,13 @@ void process_message(event_loop *loop,
|
||||||
print_worker_info("Reconstructing", state->algorithm_state);
|
print_worker_info("Reconstructing", state->algorithm_state);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
object_id *obj_id = (object_id *) utarray_front(state->input_buffer);
|
ObjectID *obj_id = (ObjectID *) utarray_front(state->input_buffer);
|
||||||
reconstruct_object(state, *obj_id);
|
reconstruct_object(state, *obj_id);
|
||||||
} break;
|
} break;
|
||||||
case DISCONNECT_CLIENT: {
|
case DISCONNECT_CLIENT: {
|
||||||
LOG_INFO("Disconnecting client on fd %d", client_sock);
|
LOG_INFO("Disconnecting client on fd %d", client_sock);
|
||||||
kill_worker(worker, false);
|
kill_worker(worker, false);
|
||||||
if (!actor_ids_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
if (!ActorID_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
||||||
/* Let the scheduling algorithm process the absence of this worker. */
|
/* Let the scheduling algorithm process the absence of this worker. */
|
||||||
handle_actor_worker_disconnect(state, state->algorithm_state,
|
handle_actor_worker_disconnect(state, state->algorithm_state,
|
||||||
worker->actor_id);
|
worker->actor_id);
|
||||||
|
@ -743,7 +743,7 @@ void process_message(event_loop *loop,
|
||||||
case NOTIFY_UNBLOCKED: {
|
case NOTIFY_UNBLOCKED: {
|
||||||
if (worker->task_in_progress != NULL) {
|
if (worker->task_in_progress != NULL) {
|
||||||
/* TODO(swang): For now, we don't handle blocked actors. */
|
/* TODO(swang): For now, we don't handle blocked actors. */
|
||||||
if (actor_ids_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
if (ActorID_equal(worker->actor_id, NIL_ACTOR_ID)) {
|
||||||
/* If the worker was executing a task (i.e. non-driver), update its
|
/* If the worker was executing a task (i.e. non-driver), update its
|
||||||
* state to not blocked. */
|
* state to not blocked. */
|
||||||
CHECK(worker->is_blocked);
|
CHECK(worker->is_blocked);
|
||||||
|
@ -762,11 +762,11 @@ void new_client_connection(event_loop *loop,
|
||||||
int listener_sock,
|
int listener_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
local_scheduler_state *state = context;
|
LocalSchedulerState *state = context;
|
||||||
int new_socket = accept_client(listener_sock);
|
int new_socket = accept_client(listener_sock);
|
||||||
/* Create a struct for this worker. This will be freed when we free the local
|
/* Create a struct for this worker. This will be freed when we free the local
|
||||||
* scheduler state. */
|
* scheduler state. */
|
||||||
local_scheduler_client *worker = malloc(sizeof(local_scheduler_client));
|
LocalSchedulerClient *worker = malloc(sizeof(LocalSchedulerClient));
|
||||||
worker->sock = new_socket;
|
worker->sock = new_socket;
|
||||||
worker->task_in_progress = NULL;
|
worker->task_in_progress = NULL;
|
||||||
worker->is_blocked = false;
|
worker->is_blocked = false;
|
||||||
|
@ -782,21 +782,21 @@ void new_client_connection(event_loop *loop,
|
||||||
|
|
||||||
/* We need this code so we can clean up when we get a SIGTERM signal. */
|
/* We need this code so we can clean up when we get a SIGTERM signal. */
|
||||||
|
|
||||||
local_scheduler_state *g_state;
|
LocalSchedulerState *g_state;
|
||||||
|
|
||||||
void signal_handler(int signal) {
|
void signal_handler(int signal) {
|
||||||
LOG_DEBUG("Signal was %d", signal);
|
LOG_DEBUG("Signal was %d", signal);
|
||||||
if (signal == SIGTERM) {
|
if (signal == SIGTERM) {
|
||||||
free_local_scheduler(g_state);
|
LocalSchedulerState_free(g_state);
|
||||||
exit(0);
|
exit(0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* End of the cleanup code. */
|
/* End of the cleanup code. */
|
||||||
|
|
||||||
void handle_task_scheduled_callback(task *original_task, void *user_context) {
|
void handle_task_scheduled_callback(Task *original_task, void *user_context) {
|
||||||
task_spec *spec = task_task_spec(original_task);
|
task_spec *spec = Task_task_spec(original_task);
|
||||||
if (actor_ids_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) {
|
if (ActorID_equal(task_spec_actor_id(spec), NIL_ACTOR_ID)) {
|
||||||
/* This task does not involve an actor. Handle it normally. */
|
/* 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);
|
||||||
} else {
|
} else {
|
||||||
|
@ -817,10 +817,10 @@ void handle_task_scheduled_callback(task *original_task, void *user_context) {
|
||||||
* for creating the actor.
|
* for creating the actor.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void handle_actor_creation_callback(actor_info info, void *context) {
|
void handle_actor_creation_callback(ActorInfo info, void *context) {
|
||||||
actor_id actor_id = info.actor_id;
|
ActorID actor_id = info.actor_id;
|
||||||
db_client_id local_scheduler_id = info.local_scheduler_id;
|
DBClientID local_scheduler_id = info.local_scheduler_id;
|
||||||
local_scheduler_state *state = context;
|
LocalSchedulerState *state = context;
|
||||||
/* Make sure the actor entry is not already present in the actor map table.
|
/* Make sure the actor entry is not already present in the actor map table.
|
||||||
* TODO(rkn): We will need to remove this check to handle the case where the
|
* TODO(rkn): We will need to remove this check to handle the case where the
|
||||||
* corresponding publish is retried and the case in which a task that creates
|
* corresponding publish is retried and the case in which a task that creates
|
||||||
|
@ -837,7 +837,7 @@ void handle_actor_creation_callback(actor_info info, void *context) {
|
||||||
HASH_ADD(hh, state->actor_mapping, actor_id, sizeof(entry->actor_id), entry);
|
HASH_ADD(hh, state->actor_mapping, actor_id, sizeof(entry->actor_id), entry);
|
||||||
/* If this local scheduler is responsible for the actor, then start a new
|
/* If this local scheduler is responsible for the actor, then start a new
|
||||||
* worker for the actor. */
|
* worker for the actor. */
|
||||||
if (db_client_ids_equal(local_scheduler_id, get_db_client_id(state->db))) {
|
if (DBClientID_equal(local_scheduler_id, get_db_client_id(state->db))) {
|
||||||
start_worker(state, actor_id);
|
start_worker(state, actor_id);
|
||||||
}
|
}
|
||||||
/* Let the scheduling algorithm process the fact that a new actor has been
|
/* Let the scheduling algorithm process the fact that a new actor has been
|
||||||
|
@ -846,9 +846,9 @@ void handle_actor_creation_callback(actor_info info, void *context) {
|
||||||
}
|
}
|
||||||
|
|
||||||
int heartbeat_handler(event_loop *loop, timer_id id, void *context) {
|
int heartbeat_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
local_scheduler_state *state = context;
|
LocalSchedulerState *state = context;
|
||||||
scheduling_algorithm_state *algorithm_state = state->algorithm_state;
|
SchedulingAlgorithmState *algorithm_state = state->algorithm_state;
|
||||||
local_scheduler_info info;
|
LocalSchedulerInfo info;
|
||||||
/* Ask the scheduling algorithm to fill out the scheduler info struct. */
|
/* Ask the scheduling algorithm to fill out the scheduler info struct. */
|
||||||
provide_scheduler_info(state, algorithm_state, &info);
|
provide_scheduler_info(state, algorithm_state, &info);
|
||||||
/* Publish the heartbeat to all subscribers of the local scheduler table. */
|
/* Publish the heartbeat to all subscribers of the local scheduler table. */
|
||||||
|
@ -873,7 +873,7 @@ void start_server(const char *node_ip_address,
|
||||||
signal(SIGPIPE, SIG_IGN);
|
signal(SIGPIPE, SIG_IGN);
|
||||||
int fd = bind_ipc_sock(socket_name, true);
|
int fd = bind_ipc_sock(socket_name, true);
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
g_state = init_local_scheduler(
|
g_state = LocalSchedulerState_init(
|
||||||
node_ip_address, loop, redis_addr, redis_port, socket_name,
|
node_ip_address, loop, redis_addr, redis_port, socket_name,
|
||||||
plasma_store_socket_name, plasma_manager_socket_name,
|
plasma_store_socket_name, plasma_manager_socket_name,
|
||||||
plasma_manager_address, global_scheduler_exists, static_resource_conf,
|
plasma_manager_address, global_scheduler_exists, static_resource_conf,
|
||||||
|
|
|
@ -38,9 +38,9 @@ void new_client_connection(event_loop *loop,
|
||||||
* @param worker The worker to assign the task to.
|
* @param worker The worker to assign the task to.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void assign_task_to_worker(local_scheduler_state *state,
|
void assign_task_to_worker(LocalSchedulerState *state,
|
||||||
task_spec *task,
|
task_spec *task,
|
||||||
local_scheduler_client *worker);
|
LocalSchedulerClient *worker);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is the callback that is used to process a notification from the Plasma
|
* This is the callback that is used to process a notification from the Plasma
|
||||||
|
@ -67,9 +67,9 @@ void process_plasma_notification(event_loop *loop,
|
||||||
* @param object_id The ID of the object to reconstruct.
|
* @param object_id The ID of the object to reconstruct.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void reconstruct_object(local_scheduler_state *state, object_id object_id);
|
void reconstruct_object(LocalSchedulerState *state, ObjectID object_id);
|
||||||
|
|
||||||
void print_resource_info(const local_scheduler_state *s, const task_spec *spec);
|
void print_resource_info(const LocalSchedulerState *s, const task_spec *spec);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Kill a worker.
|
* Kill a worker.
|
||||||
|
@ -79,7 +79,7 @@ void print_resource_info(const local_scheduler_state *s, const task_spec *spec);
|
||||||
* exit.
|
* exit.
|
||||||
* @param Void.
|
* @param Void.
|
||||||
*/
|
*/
|
||||||
void kill_worker(local_scheduler_client *worker, bool wait);
|
void kill_worker(LocalSchedulerClient *worker, bool wait);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Start a worker. This forks a new worker process that can be added to the
|
* Start a worker. This forks a new worker process that can be added to the
|
||||||
|
@ -91,7 +91,7 @@ void kill_worker(local_scheduler_client *worker, bool wait);
|
||||||
* actor, then NIL_ACTOR_ID should be used.
|
* actor, then NIL_ACTOR_ID should be used.
|
||||||
* @param Void.
|
* @param Void.
|
||||||
*/
|
*/
|
||||||
void start_worker(local_scheduler_state *state, actor_id actor_id);
|
void start_worker(LocalSchedulerState *state, ActorID actor_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update our accounting for the current resources being used, according to
|
* Update our accounting for the current resources being used, according to
|
||||||
|
@ -105,13 +105,13 @@ void start_worker(local_scheduler_state *state, actor_id actor_id);
|
||||||
* available. Else, it will take from the dynamic resources available.
|
* available. Else, it will take from the dynamic resources available.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void update_dynamic_resources(local_scheduler_state *state,
|
void update_dynamic_resources(LocalSchedulerState *state,
|
||||||
task_spec *spec,
|
task_spec *spec,
|
||||||
bool return_resources);
|
bool return_resources);
|
||||||
|
|
||||||
/** The following methods are for testing purposes only. */
|
/** The following methods are for testing purposes only. */
|
||||||
#ifdef PHOTON_TEST
|
#ifdef PHOTON_TEST
|
||||||
local_scheduler_state *init_local_scheduler(
|
LocalSchedulerState *LocalSchedulerState_init(
|
||||||
const char *node_ip_address,
|
const char *node_ip_address,
|
||||||
event_loop *loop,
|
event_loop *loop,
|
||||||
const char *redis_addr,
|
const char *redis_addr,
|
||||||
|
@ -125,9 +125,9 @@ local_scheduler_state *init_local_scheduler(
|
||||||
const char *worker_path,
|
const char *worker_path,
|
||||||
int num_workers);
|
int num_workers);
|
||||||
|
|
||||||
void free_local_scheduler(local_scheduler_state *state);
|
void LocalSchedulerState_free(LocalSchedulerState *state);
|
||||||
|
|
||||||
scheduling_algorithm_state *get_algorithm_state(local_scheduler_state *state);
|
SchedulingAlgorithmState *get_algorithm_state(LocalSchedulerState *state);
|
||||||
|
|
||||||
void process_message(event_loop *loop,
|
void process_message(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
|
|
|
@ -41,23 +41,23 @@ typedef struct {
|
||||||
/** Photon's socket for IPC requests. */
|
/** Photon's socket for IPC requests. */
|
||||||
int photon_fd;
|
int photon_fd;
|
||||||
/** Photon's local scheduler state. */
|
/** Photon's local scheduler state. */
|
||||||
local_scheduler_state *photon_state;
|
LocalSchedulerState *photon_state;
|
||||||
/** Photon's event loop. */
|
/** Photon's event loop. */
|
||||||
event_loop *loop;
|
event_loop *loop;
|
||||||
/** Number of Photon client connections, or mock workers. */
|
/** Number of Photon client connections, or mock workers. */
|
||||||
int num_photon_conns;
|
int num_photon_conns;
|
||||||
/** Photon client connections. */
|
/** Photon client connections. */
|
||||||
photon_conn **conns;
|
PhotonConnection **conns;
|
||||||
} photon_mock;
|
} PhotonMock;
|
||||||
|
|
||||||
photon_mock *init_photon_mock(int num_workers, int num_mock_workers) {
|
PhotonMock *PhotonMock_init(int num_workers, int num_mock_workers) {
|
||||||
const char *node_ip_address = "127.0.0.1";
|
const char *node_ip_address = "127.0.0.1";
|
||||||
const char *redis_addr = node_ip_address;
|
const char *redis_addr = node_ip_address;
|
||||||
int redis_port = 6379;
|
int redis_port = 6379;
|
||||||
const double static_resource_conf[MAX_RESOURCE_INDEX] = {DEFAULT_NUM_CPUS,
|
const double static_resource_conf[MAX_RESOURCE_INDEX] = {DEFAULT_NUM_CPUS,
|
||||||
DEFAULT_NUM_GPUS};
|
DEFAULT_NUM_GPUS};
|
||||||
photon_mock *mock = malloc(sizeof(photon_mock));
|
PhotonMock *mock = malloc(sizeof(PhotonMock));
|
||||||
memset(mock, 0, sizeof(photon_mock));
|
memset(mock, 0, sizeof(PhotonMock));
|
||||||
mock->loop = event_loop_create();
|
mock->loop = event_loop_create();
|
||||||
/* Bind to the Photon port and initialize the Photon scheduler. */
|
/* Bind to the Photon port and initialize the Photon scheduler. */
|
||||||
UT_string *plasma_manager_socket_name = bind_ipc_sock_retry(
|
UT_string *plasma_manager_socket_name = bind_ipc_sock_retry(
|
||||||
|
@ -79,7 +79,7 @@ photon_mock *init_photon_mock(int num_workers, int num_mock_workers) {
|
||||||
utstring_body(plasma_manager_socket_name),
|
utstring_body(plasma_manager_socket_name),
|
||||||
utstring_body(photon_socket_name), redis_addr, redis_port);
|
utstring_body(photon_socket_name), redis_addr, redis_port);
|
||||||
|
|
||||||
mock->photon_state = init_local_scheduler(
|
mock->photon_state = LocalSchedulerState_init(
|
||||||
"127.0.0.1", mock->loop, redis_addr, redis_port,
|
"127.0.0.1", mock->loop, redis_addr, redis_port,
|
||||||
utstring_body(photon_socket_name), plasma_store_socket_name,
|
utstring_body(photon_socket_name), plasma_store_socket_name,
|
||||||
utstring_body(plasma_manager_socket_name), NULL, false,
|
utstring_body(plasma_manager_socket_name), NULL, false,
|
||||||
|
@ -92,10 +92,10 @@ photon_mock *init_photon_mock(int num_workers, int num_mock_workers) {
|
||||||
|
|
||||||
/* Connect a Photon client. */
|
/* Connect a Photon client. */
|
||||||
mock->num_photon_conns = num_mock_workers;
|
mock->num_photon_conns = num_mock_workers;
|
||||||
mock->conns = malloc(sizeof(photon_conn *) * num_mock_workers);
|
mock->conns = malloc(sizeof(PhotonConnection *) * num_mock_workers);
|
||||||
for (int i = 0; i < num_mock_workers; ++i) {
|
for (int i = 0; i < num_mock_workers; ++i) {
|
||||||
mock->conns[i] =
|
mock->conns[i] =
|
||||||
photon_connect(utstring_body(photon_socket_name), NIL_ACTOR_ID);
|
PhotonConnection_init(utstring_body(photon_socket_name), NIL_ACTOR_ID);
|
||||||
new_client_connection(mock->loop, mock->photon_fd,
|
new_client_connection(mock->loop, mock->photon_fd,
|
||||||
(void *) mock->photon_state, 0);
|
(void *) mock->photon_state, 0);
|
||||||
}
|
}
|
||||||
|
@ -106,20 +106,20 @@ photon_mock *init_photon_mock(int num_workers, int num_mock_workers) {
|
||||||
return mock;
|
return mock;
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_photon_mock(photon_mock *mock) {
|
void PhotonMock_free(PhotonMock *mock) {
|
||||||
/* Disconnect clients. */
|
/* Disconnect clients. */
|
||||||
for (int i = 0; i < mock->num_photon_conns; ++i) {
|
for (int i = 0; i < mock->num_photon_conns; ++i) {
|
||||||
photon_disconnect(mock->conns[i]);
|
PhotonConnection_free(mock->conns[i]);
|
||||||
}
|
}
|
||||||
free(mock->conns);
|
free(mock->conns);
|
||||||
|
|
||||||
/* Kill all the workers and run the event loop again so that the task table
|
/* Kill all the workers and run the event loop again so that the task table
|
||||||
* updates propagate and the tasks in progress are freed. */
|
* updates propagate and the tasks in progress are freed. */
|
||||||
local_scheduler_client **worker = (local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient **worker =
|
||||||
mock->photon_state->workers, 0);
|
(LocalSchedulerClient **) utarray_eltptr(mock->photon_state->workers, 0);
|
||||||
while (worker != NULL) {
|
while (worker != NULL) {
|
||||||
kill_worker(*worker, true);
|
kill_worker(*worker, true);
|
||||||
worker = (local_scheduler_client **) utarray_eltptr(
|
worker = (LocalSchedulerClient **) utarray_eltptr(
|
||||||
mock->photon_state->workers, 0);
|
mock->photon_state->workers, 0);
|
||||||
}
|
}
|
||||||
event_loop_add_timer(mock->loop, 500,
|
event_loop_add_timer(mock->loop, 500,
|
||||||
|
@ -127,15 +127,15 @@ void destroy_photon_mock(photon_mock *mock) {
|
||||||
event_loop_run(mock->loop);
|
event_loop_run(mock->loop);
|
||||||
|
|
||||||
/* This also frees mock->loop. */
|
/* This also frees mock->loop. */
|
||||||
free_local_scheduler(mock->photon_state);
|
LocalSchedulerState_free(mock->photon_state);
|
||||||
close(mock->plasma_store_fd);
|
close(mock->plasma_store_fd);
|
||||||
close(mock->plasma_manager_fd);
|
close(mock->plasma_manager_fd);
|
||||||
free(mock);
|
free(mock);
|
||||||
}
|
}
|
||||||
|
|
||||||
void reset_worker(photon_mock *mock, local_scheduler_client *worker) {
|
void reset_worker(PhotonMock *mock, LocalSchedulerClient *worker) {
|
||||||
if (worker->task_in_progress) {
|
if (worker->task_in_progress) {
|
||||||
free_task(worker->task_in_progress);
|
Task_free(worker->task_in_progress);
|
||||||
worker->task_in_progress = NULL;
|
worker->task_in_progress = NULL;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -146,12 +146,12 @@ void reset_worker(photon_mock *mock, local_scheduler_client *worker) {
|
||||||
* value, the task should get assigned to a worker again.
|
* value, the task should get assigned to a worker again.
|
||||||
*/
|
*/
|
||||||
TEST object_reconstruction_test(void) {
|
TEST object_reconstruction_test(void) {
|
||||||
photon_mock *photon = init_photon_mock(0, 1);
|
PhotonMock *photon = PhotonMock_init(0, 1);
|
||||||
photon_conn *worker = photon->conns[0];
|
PhotonConnection *worker = photon->conns[0];
|
||||||
|
|
||||||
/* Create a task with zero dependencies and one return value. */
|
/* Create a task with zero dependencies and one return value. */
|
||||||
task_spec *spec = example_task_spec(0, 1);
|
task_spec *spec = example_task_spec(0, 1);
|
||||||
object_id return_id = task_return(spec, 0);
|
ObjectID return_id = task_return(spec, 0);
|
||||||
|
|
||||||
/* Add an empty object table entry for the object we want to reconstruct, to
|
/* Add an empty object table entry for the object we want to reconstruct, to
|
||||||
* simulate it having been created and evicted. */
|
* simulate it having been created and evicted. */
|
||||||
|
@ -179,7 +179,7 @@ TEST object_reconstruction_test(void) {
|
||||||
free_task_spec(reconstruct_task);
|
free_task_spec(reconstruct_task);
|
||||||
free_task_spec(task_assigned);
|
free_task_spec(task_assigned);
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
exit(0);
|
exit(0);
|
||||||
} else {
|
} else {
|
||||||
/* Run the event loop. NOTE: OSX appears to require the parent process to
|
/* Run the event loop. NOTE: OSX appears to require the parent process to
|
||||||
|
@ -189,11 +189,11 @@ TEST object_reconstruction_test(void) {
|
||||||
event_loop_run(photon->loop);
|
event_loop_run(photon->loop);
|
||||||
/* Set the task's status to TASK_STATUS_DONE to prevent the race condition
|
/* Set the task's status to TASK_STATUS_DONE to prevent the race condition
|
||||||
* that would suppress object reconstruction. */
|
* that would suppress object reconstruction. */
|
||||||
task *task = alloc_task(spec, TASK_STATUS_DONE,
|
Task *task = Task_alloc(spec, TASK_STATUS_DONE,
|
||||||
get_db_client_id(photon->photon_state->db));
|
get_db_client_id(photon->photon_state->db));
|
||||||
task_table_add_task(photon->photon_state->db, task, NULL, NULL, NULL);
|
task_table_add_task(photon->photon_state->db, task, NULL, NULL, NULL);
|
||||||
/* Trigger reconstruction, and run the event loop again. */
|
/* Trigger reconstruction, and run the event loop again. */
|
||||||
object_id return_id = task_return(spec, 0);
|
ObjectID return_id = task_return(spec, 0);
|
||||||
photon_reconstruct_object(worker, return_id);
|
photon_reconstruct_object(worker, return_id);
|
||||||
event_loop_add_timer(photon->loop, 500,
|
event_loop_add_timer(photon->loop, 500,
|
||||||
(event_loop_timer_handler) timeout_handler, NULL);
|
(event_loop_timer_handler) timeout_handler, NULL);
|
||||||
|
@ -204,7 +204,7 @@ TEST object_reconstruction_test(void) {
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
ASSERT_EQ(num_waiting_tasks(photon->photon_state->algorithm_state), 0);
|
ASSERT_EQ(num_waiting_tasks(photon->photon_state->algorithm_state), 0);
|
||||||
ASSERT_EQ(num_dispatch_tasks(photon->photon_state->algorithm_state), 0);
|
ASSERT_EQ(num_dispatch_tasks(photon->photon_state->algorithm_state), 0);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -215,15 +215,15 @@ TEST object_reconstruction_test(void) {
|
||||||
* should trigger reconstruction of all previous tasks in the lineage.
|
* should trigger reconstruction of all previous tasks in the lineage.
|
||||||
*/
|
*/
|
||||||
TEST object_reconstruction_recursive_test(void) {
|
TEST object_reconstruction_recursive_test(void) {
|
||||||
photon_mock *photon = init_photon_mock(0, 1);
|
PhotonMock *photon = PhotonMock_init(0, 1);
|
||||||
photon_conn *worker = photon->conns[0];
|
PhotonConnection *worker = photon->conns[0];
|
||||||
/* Create a chain of tasks, each one dependent on the one before it. Mark
|
/* Create a chain of tasks, each one dependent on the one before it. Mark
|
||||||
* each object as available so that tasks will run immediately. */
|
* each object as available so that tasks will run immediately. */
|
||||||
const int NUM_TASKS = 10;
|
const int NUM_TASKS = 10;
|
||||||
task_spec *specs[NUM_TASKS];
|
task_spec *specs[NUM_TASKS];
|
||||||
specs[0] = example_task_spec(0, 1);
|
specs[0] = example_task_spec(0, 1);
|
||||||
for (int i = 1; i < NUM_TASKS; ++i) {
|
for (int i = 1; i < NUM_TASKS; ++i) {
|
||||||
object_id arg_id = task_return(specs[i - 1], 0);
|
ObjectID arg_id = task_return(specs[i - 1], 0);
|
||||||
handle_object_available(photon->photon_state,
|
handle_object_available(photon->photon_state,
|
||||||
photon->photon_state->algorithm_state, arg_id);
|
photon->photon_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);
|
||||||
|
@ -234,7 +234,7 @@ TEST object_reconstruction_recursive_test(void) {
|
||||||
const char *client_id = "clientid";
|
const char *client_id = "clientid";
|
||||||
redisContext *context = redisConnect("127.0.0.1", 6379);
|
redisContext *context = redisConnect("127.0.0.1", 6379);
|
||||||
for (int i = 0; i < NUM_TASKS; ++i) {
|
for (int i = 0; i < NUM_TASKS; ++i) {
|
||||||
object_id return_id = task_return(specs[i], 0);
|
ObjectID return_id = task_return(specs[i], 0);
|
||||||
redisReply *reply = redisCommand(
|
redisReply *reply = redisCommand(
|
||||||
context, "RAY.OBJECT_TABLE_ADD %b %ld %b %s", return_id.id,
|
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);
|
sizeof(return_id.id), 1, NIL_DIGEST, (size_t) DIGEST_SIZE, client_id);
|
||||||
|
@ -277,7 +277,7 @@ TEST object_reconstruction_recursive_test(void) {
|
||||||
free_task_spec(task_assigned);
|
free_task_spec(task_assigned);
|
||||||
ASSERT(found);
|
ASSERT(found);
|
||||||
}
|
}
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
exit(0);
|
exit(0);
|
||||||
} else {
|
} else {
|
||||||
/* Run the event loop. NOTE: OSX appears to require the parent process to
|
/* Run the event loop. NOTE: OSX appears to require the parent process to
|
||||||
|
@ -287,12 +287,12 @@ TEST object_reconstruction_recursive_test(void) {
|
||||||
event_loop_run(photon->loop);
|
event_loop_run(photon->loop);
|
||||||
/* Set the final task's status to TASK_STATUS_DONE to prevent the race
|
/* Set the final task's status to TASK_STATUS_DONE to prevent the race
|
||||||
* condition that would suppress object reconstruction. */
|
* condition that would suppress object reconstruction. */
|
||||||
task *last_task = alloc_task(specs[NUM_TASKS - 1], TASK_STATUS_DONE,
|
Task *last_task = Task_alloc(specs[NUM_TASKS - 1], TASK_STATUS_DONE,
|
||||||
get_db_client_id(photon->photon_state->db));
|
get_db_client_id(photon->photon_state->db));
|
||||||
task_table_add_task(photon->photon_state->db, last_task, NULL, NULL, NULL);
|
task_table_add_task(photon->photon_state->db, last_task, NULL, NULL, NULL);
|
||||||
/* Trigger reconstruction for the last object, and run the event loop
|
/* Trigger reconstruction for the last object, and run the event loop
|
||||||
* again. */
|
* again. */
|
||||||
object_id return_id = task_return(specs[NUM_TASKS - 1], 0);
|
ObjectID return_id = task_return(specs[NUM_TASKS - 1], 0);
|
||||||
photon_reconstruct_object(worker, return_id);
|
photon_reconstruct_object(worker, return_id);
|
||||||
event_loop_add_timer(photon->loop, 500,
|
event_loop_add_timer(photon->loop, 500,
|
||||||
(event_loop_timer_handler) timeout_handler, NULL);
|
(event_loop_timer_handler) timeout_handler, NULL);
|
||||||
|
@ -305,7 +305,7 @@ TEST object_reconstruction_recursive_test(void) {
|
||||||
for (int i = 0; i < NUM_TASKS; ++i) {
|
for (int i = 0; i < NUM_TASKS; ++i) {
|
||||||
free_task_spec(specs[i]);
|
free_task_spec(specs[i]);
|
||||||
}
|
}
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -316,19 +316,19 @@ TEST object_reconstruction_recursive_test(void) {
|
||||||
*/
|
*/
|
||||||
task_spec *object_reconstruction_suppression_spec;
|
task_spec *object_reconstruction_suppression_spec;
|
||||||
|
|
||||||
void object_reconstruction_suppression_callback(object_id object_id,
|
void object_reconstruction_suppression_callback(ObjectID object_id,
|
||||||
void *user_context) {
|
void *user_context) {
|
||||||
/* Submit the task after adding the object to the object table. */
|
/* Submit the task after adding the object to the object table. */
|
||||||
photon_conn *worker = user_context;
|
PhotonConnection *worker = user_context;
|
||||||
photon_submit(worker, object_reconstruction_suppression_spec);
|
photon_submit(worker, object_reconstruction_suppression_spec);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST object_reconstruction_suppression_test(void) {
|
TEST object_reconstruction_suppression_test(void) {
|
||||||
photon_mock *photon = init_photon_mock(0, 1);
|
PhotonMock *photon = PhotonMock_init(0, 1);
|
||||||
photon_conn *worker = photon->conns[0];
|
PhotonConnection *worker = photon->conns[0];
|
||||||
|
|
||||||
object_reconstruction_suppression_spec = example_task_spec(0, 1);
|
object_reconstruction_suppression_spec = example_task_spec(0, 1);
|
||||||
object_id return_id = task_return(object_reconstruction_suppression_spec, 0);
|
ObjectID return_id = task_return(object_reconstruction_suppression_spec, 0);
|
||||||
pid_t pid = fork();
|
pid_t pid = fork();
|
||||||
if (pid == 0) {
|
if (pid == 0) {
|
||||||
/* Make sure we receive the task once. This will block until the
|
/* Make sure we receive the task once. This will block until the
|
||||||
|
@ -343,13 +343,13 @@ TEST object_reconstruction_suppression_test(void) {
|
||||||
/* Clean up. */
|
/* Clean up. */
|
||||||
free_task_spec(task_assigned);
|
free_task_spec(task_assigned);
|
||||||
free_task_spec(object_reconstruction_suppression_spec);
|
free_task_spec(object_reconstruction_suppression_spec);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
exit(0);
|
exit(0);
|
||||||
} else {
|
} else {
|
||||||
/* Connect a plasma manager client so we can call object_table_add. */
|
/* Connect a plasma manager client so we can call object_table_add. */
|
||||||
const char *db_connect_args[] = {"address", "127.0.0.1:12346"};
|
const char *db_connect_args[] = {"address", "127.0.0.1:12346"};
|
||||||
db_handle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
DBHandle *db = db_connect("127.0.0.1", 6379, "plasma_manager", "127.0.0.1",
|
||||||
2, db_connect_args);
|
2, db_connect_args);
|
||||||
db_attach(db, photon->loop, false);
|
db_attach(db, photon->loop, false);
|
||||||
/* Add the object to the object table. */
|
/* Add the object to the object table. */
|
||||||
object_table_add(db, return_id, 1, (unsigned char *) NIL_DIGEST, NULL,
|
object_table_add(db, return_id, 1, (unsigned char *) NIL_DIGEST, NULL,
|
||||||
|
@ -367,20 +367,20 @@ TEST object_reconstruction_suppression_test(void) {
|
||||||
ASSERT_EQ(num_dispatch_tasks(photon->photon_state->algorithm_state), 0);
|
ASSERT_EQ(num_dispatch_tasks(photon->photon_state->algorithm_state), 0);
|
||||||
free_task_spec(object_reconstruction_suppression_spec);
|
free_task_spec(object_reconstruction_suppression_spec);
|
||||||
db_disconnect(db);
|
db_disconnect(db);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST task_dependency_test(void) {
|
TEST task_dependency_test(void) {
|
||||||
photon_mock *photon = init_photon_mock(0, 1);
|
PhotonMock *photon = PhotonMock_init(0, 1);
|
||||||
local_scheduler_state *state = photon->photon_state;
|
LocalSchedulerState *state = photon->photon_state;
|
||||||
scheduling_algorithm_state *algorithm_state = state->algorithm_state;
|
SchedulingAlgorithmState *algorithm_state = state->algorithm_state;
|
||||||
/* Get the first worker. */
|
/* Get the first worker. */
|
||||||
local_scheduler_client *worker =
|
LocalSchedulerClient *worker =
|
||||||
*((local_scheduler_client **) utarray_eltptr(state->workers, 0));
|
*((LocalSchedulerClient **) utarray_eltptr(state->workers, 0));
|
||||||
task_spec *spec = example_task_spec(1, 1);
|
task_spec *spec = example_task_spec(1, 1);
|
||||||
object_id oid = task_arg_id(spec, 0);
|
ObjectID oid = task_arg_id(spec, 0);
|
||||||
|
|
||||||
/* Check that the task gets queued in the waiting queue if the task is
|
/* Check that the task gets queued in the waiting queue if the task is
|
||||||
* submitted, but the input and workers are not available. */
|
* submitted, but the input and workers are not available. */
|
||||||
|
@ -443,20 +443,20 @@ TEST task_dependency_test(void) {
|
||||||
ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0);
|
ASSERT_EQ(num_dispatch_tasks(algorithm_state), 0);
|
||||||
|
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST task_multi_dependency_test(void) {
|
TEST task_multi_dependency_test(void) {
|
||||||
photon_mock *photon = init_photon_mock(0, 1);
|
PhotonMock *photon = PhotonMock_init(0, 1);
|
||||||
local_scheduler_state *state = photon->photon_state;
|
LocalSchedulerState *state = photon->photon_state;
|
||||||
scheduling_algorithm_state *algorithm_state = state->algorithm_state;
|
SchedulingAlgorithmState *algorithm_state = state->algorithm_state;
|
||||||
/* Get the first worker. */
|
/* Get the first worker. */
|
||||||
local_scheduler_client *worker =
|
LocalSchedulerClient *worker =
|
||||||
*((local_scheduler_client **) utarray_eltptr(state->workers, 0));
|
*((LocalSchedulerClient **) utarray_eltptr(state->workers, 0));
|
||||||
task_spec *spec = example_task_spec(2, 1);
|
task_spec *spec = example_task_spec(2, 1);
|
||||||
object_id oid1 = task_arg_id(spec, 0);
|
ObjectID oid1 = task_arg_id(spec, 0);
|
||||||
object_id oid2 = task_arg_id(spec, 1);
|
ObjectID oid2 = task_arg_id(spec, 1);
|
||||||
|
|
||||||
/* Check that the task gets queued in the waiting queue if the task is
|
/* Check that the task gets queued in the waiting queue if the task is
|
||||||
* submitted, but the inputs and workers are not available. */
|
* submitted, but the inputs and workers are not available. */
|
||||||
|
@ -517,14 +517,14 @@ TEST task_multi_dependency_test(void) {
|
||||||
reset_worker(photon, worker);
|
reset_worker(photon, worker);
|
||||||
|
|
||||||
free_task_spec(spec);
|
free_task_spec(spec);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST start_kill_workers_test(void) {
|
TEST start_kill_workers_test(void) {
|
||||||
/* Start some workers. */
|
/* Start some workers. */
|
||||||
int num_workers = 4;
|
int num_workers = 4;
|
||||||
photon_mock *photon = init_photon_mock(num_workers, 0);
|
PhotonMock *photon = PhotonMock_init(num_workers, 0);
|
||||||
/* We start off with num_workers children processes, but no workers
|
/* We start off with num_workers children processes, but no workers
|
||||||
* registered yet. */
|
* registered yet. */
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), num_workers);
|
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), num_workers);
|
||||||
|
@ -545,16 +545,15 @@ TEST start_kill_workers_test(void) {
|
||||||
|
|
||||||
/* Each worker should register its process ID. */
|
/* Each worker should register its process ID. */
|
||||||
for (int i = 0; i < utarray_len(photon->photon_state->workers); ++i) {
|
for (int i = 0; i < utarray_len(photon->photon_state->workers); ++i) {
|
||||||
local_scheduler_client *worker =
|
LocalSchedulerClient *worker = *(LocalSchedulerClient **) utarray_eltptr(
|
||||||
*(local_scheduler_client **) utarray_eltptr(
|
photon->photon_state->workers, i);
|
||||||
photon->photon_state->workers, i);
|
|
||||||
process_message(photon->photon_state->loop, worker->sock, worker, 0);
|
process_message(photon->photon_state->loop, worker->sock, worker, 0);
|
||||||
}
|
}
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->workers), num_workers);
|
ASSERT_EQ(utarray_len(photon->photon_state->workers), num_workers);
|
||||||
|
|
||||||
/* After killing a worker, its state is cleaned up. */
|
/* After killing a worker, its state is cleaned up. */
|
||||||
local_scheduler_client *worker = *(local_scheduler_client **) utarray_eltptr(
|
LocalSchedulerClient *worker = *(LocalSchedulerClient **) utarray_eltptr(
|
||||||
photon->photon_state->workers, 0);
|
photon->photon_state->workers, 0);
|
||||||
kill_worker(worker, false);
|
kill_worker(worker, false);
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
||||||
|
@ -573,7 +572,7 @@ TEST start_kill_workers_test(void) {
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 1);
|
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 1);
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->workers), num_workers);
|
ASSERT_EQ(utarray_len(photon->photon_state->workers), num_workers);
|
||||||
/* Make sure that the new worker registers its process ID. */
|
/* Make sure that the new worker registers its process ID. */
|
||||||
worker = *(local_scheduler_client **) utarray_eltptr(
|
worker = *(LocalSchedulerClient **) utarray_eltptr(
|
||||||
photon->photon_state->workers, num_workers - 1);
|
photon->photon_state->workers, num_workers - 1);
|
||||||
process_message(photon->photon_state->loop, worker->sock, worker, 0);
|
process_message(photon->photon_state->loop, worker->sock, worker, 0);
|
||||||
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
ASSERT_EQ(utarray_len(photon->photon_state->child_pids), 0);
|
||||||
|
@ -581,7 +580,7 @@ TEST start_kill_workers_test(void) {
|
||||||
|
|
||||||
/* Clean up. */
|
/* Clean up. */
|
||||||
close(new_worker_fd);
|
close(new_worker_fd);
|
||||||
destroy_photon_mock(photon);
|
PhotonMock_free(photon);
|
||||||
PASS();
|
PASS();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -6,7 +6,7 @@
|
||||||
* used to implement an LRU cache. */
|
* used to implement an LRU cache. */
|
||||||
typedef struct released_object {
|
typedef struct released_object {
|
||||||
/** The object_id of the released object. */
|
/** The object_id of the released object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Needed for the doubly-linked list macros. */
|
/** Needed for the doubly-linked list macros. */
|
||||||
struct released_object *prev;
|
struct released_object *prev;
|
||||||
/** Needed for the doubly-linked list macros. */
|
/** Needed for the doubly-linked list macros. */
|
||||||
|
@ -17,7 +17,7 @@ typedef struct released_object {
|
||||||
* object to its location in the doubly-linked list of released objects. */
|
* object to its location in the doubly-linked list of released objects. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Object ID of this object. */
|
/** Object ID of this object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** A pointer to the corresponding entry for this object in the doubly-linked
|
/** A pointer to the corresponding entry for this object in the doubly-linked
|
||||||
* list of released objects. */
|
* list of released objects. */
|
||||||
released_object *released_object;
|
released_object *released_object;
|
||||||
|
@ -26,7 +26,7 @@ typedef struct {
|
||||||
} released_object_entry;
|
} released_object_entry;
|
||||||
|
|
||||||
/** The part of the Plasma state that is maintained by the eviction policy. */
|
/** The part of the Plasma state that is maintained by the eviction policy. */
|
||||||
struct eviction_state {
|
struct EvictionState {
|
||||||
/** The amount of memory (in bytes) currently being used. */
|
/** The amount of memory (in bytes) currently being used. */
|
||||||
int64_t memory_used;
|
int64_t memory_used;
|
||||||
/** A doubly-linked list of the released objects in order from least recently
|
/** A doubly-linked list of the released objects in order from least recently
|
||||||
|
@ -39,17 +39,17 @@ struct eviction_state {
|
||||||
|
|
||||||
/* This is used to define the array of object IDs used to define the
|
/* This is used to define the array of object IDs used to define the
|
||||||
* released_objects type. */
|
* released_objects type. */
|
||||||
UT_icd released_objects_entry_icd = {sizeof(object_id), NULL, NULL, NULL};
|
UT_icd released_objects_entry_icd = {sizeof(ObjectID), NULL, NULL, NULL};
|
||||||
|
|
||||||
eviction_state *make_eviction_state(void) {
|
EvictionState *EvictionState_init(void) {
|
||||||
eviction_state *state = malloc(sizeof(eviction_state));
|
EvictionState *state = malloc(sizeof(EvictionState));
|
||||||
state->memory_used = 0;
|
state->memory_used = 0;
|
||||||
state->released_objects = NULL;
|
state->released_objects = NULL;
|
||||||
state->released_object_table = NULL;
|
state->released_object_table = NULL;
|
||||||
return state;
|
return state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void free_eviction_state(eviction_state *s) {
|
void EvictionState_free(EvictionState *s) {
|
||||||
/* Delete each element in the doubly-linked list. */
|
/* Delete each element in the doubly-linked list. */
|
||||||
released_object *element, *temp;
|
released_object *element, *temp;
|
||||||
DL_FOREACH_SAFE(s->released_objects, element, temp) {
|
DL_FOREACH_SAFE(s->released_objects, element, temp) {
|
||||||
|
@ -66,8 +66,8 @@ void free_eviction_state(eviction_state *s) {
|
||||||
free(s);
|
free(s);
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_object_to_lru_cache(eviction_state *eviction_state,
|
void add_object_to_lru_cache(EvictionState *eviction_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
/* Add the object ID to the doubly-linked list. */
|
/* Add the object ID to the doubly-linked list. */
|
||||||
released_object *linked_list_entry = malloc(sizeof(released_object));
|
released_object *linked_list_entry = malloc(sizeof(released_object));
|
||||||
linked_list_entry->object_id = object_id;
|
linked_list_entry->object_id = object_id;
|
||||||
|
@ -85,8 +85,8 @@ void add_object_to_lru_cache(eviction_state *eviction_state,
|
||||||
sizeof(object_id), hash_table_entry);
|
sizeof(object_id), hash_table_entry);
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_object_from_lru_cache(eviction_state *eviction_state,
|
void remove_object_from_lru_cache(EvictionState *eviction_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
/* Check that the object ID is in the hash table. */
|
/* Check that the object ID is in the hash table. */
|
||||||
released_object_entry *hash_table_entry;
|
released_object_entry *hash_table_entry;
|
||||||
HASH_FIND(handle, eviction_state->released_object_table, &object_id,
|
HASH_FIND(handle, eviction_state->released_object_table, &object_id,
|
||||||
|
@ -104,11 +104,12 @@ void remove_object_from_lru_cache(eviction_state *eviction_state,
|
||||||
free(hash_table_entry);
|
free(hash_table_entry);
|
||||||
}
|
}
|
||||||
|
|
||||||
int64_t choose_objects_to_evict(eviction_state *eviction_state,
|
int64_t EvictionState_choose_objects_to_evict(
|
||||||
plasma_store_info *plasma_store_info,
|
EvictionState *eviction_state,
|
||||||
int64_t num_bytes_required,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t num_bytes_required,
|
||||||
object_id **objects_to_evict) {
|
int64_t *num_objects_to_evict,
|
||||||
|
ObjectID **objects_to_evict) {
|
||||||
int64_t num_objects = 0;
|
int64_t num_objects = 0;
|
||||||
int64_t num_bytes = 0;
|
int64_t num_bytes = 0;
|
||||||
/* Figure out how many objects need to be evicted in order to recover a
|
/* Figure out how many objects need to be evicted in order to recover a
|
||||||
|
@ -131,7 +132,7 @@ int64_t choose_objects_to_evict(eviction_state *eviction_state,
|
||||||
if (num_objects == 0) {
|
if (num_objects == 0) {
|
||||||
*objects_to_evict = NULL;
|
*objects_to_evict = NULL;
|
||||||
} else {
|
} else {
|
||||||
*objects_to_evict = (object_id *) malloc(num_objects * sizeof(object_id));
|
*objects_to_evict = (ObjectID *) malloc(num_objects * sizeof(ObjectID));
|
||||||
int counter = 0;
|
int counter = 0;
|
||||||
DL_FOREACH_SAFE(eviction_state->released_objects, element, temp) {
|
DL_FOREACH_SAFE(eviction_state->released_objects, element, temp) {
|
||||||
if (counter == num_objects) {
|
if (counter == num_objects) {
|
||||||
|
@ -148,17 +149,17 @@ int64_t choose_objects_to_evict(eviction_state *eviction_state,
|
||||||
return num_bytes;
|
return num_bytes;
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_created(eviction_state *eviction_state,
|
void EvictionState_object_created(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id) {
|
ObjectID obj_id) {
|
||||||
add_object_to_lru_cache(eviction_state, obj_id);
|
add_object_to_lru_cache(eviction_state, obj_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool require_space(eviction_state *eviction_state,
|
bool EvictionState_require_space(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
int64_t size,
|
int64_t size,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict) {
|
ObjectID **objects_to_evict) {
|
||||||
/* Check if there is enough space to create the object. */
|
/* Check if there is enough space to create the object. */
|
||||||
int64_t required_space =
|
int64_t required_space =
|
||||||
eviction_state->memory_used + size - plasma_store_info->memory_capacity;
|
eviction_state->memory_used + size - plasma_store_info->memory_capacity;
|
||||||
|
@ -169,7 +170,7 @@ bool require_space(eviction_state *eviction_state,
|
||||||
int64_t space_to_free = MAX(size, plasma_store_info->memory_capacity / 5);
|
int64_t space_to_free = MAX(size, plasma_store_info->memory_capacity / 5);
|
||||||
LOG_DEBUG("not enough space to create this object, so evicting objects");
|
LOG_DEBUG("not enough space to create this object, so evicting objects");
|
||||||
/* Choose some objects to evict, and update the return pointers. */
|
/* Choose some objects to evict, and update the return pointers. */
|
||||||
num_bytes_evicted = choose_objects_to_evict(
|
num_bytes_evicted = EvictionState_choose_objects_to_evict(
|
||||||
eviction_state, plasma_store_info, space_to_free, num_objects_to_evict,
|
eviction_state, plasma_store_info, space_to_free, num_objects_to_evict,
|
||||||
objects_to_evict);
|
objects_to_evict);
|
||||||
LOG_INFO(
|
LOG_INFO(
|
||||||
|
@ -189,22 +190,22 @@ bool require_space(eviction_state *eviction_state,
|
||||||
return num_bytes_evicted >= required_space;
|
return num_bytes_evicted >= required_space;
|
||||||
}
|
}
|
||||||
|
|
||||||
void begin_object_access(eviction_state *eviction_state,
|
void EvictionState_begin_object_access(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict) {
|
ObjectID **objects_to_evict) {
|
||||||
/* If the object is in the LRU cache, remove it. */
|
/* If the object is in the LRU cache, remove it. */
|
||||||
remove_object_from_lru_cache(eviction_state, obj_id);
|
remove_object_from_lru_cache(eviction_state, obj_id);
|
||||||
*num_objects_to_evict = 0;
|
*num_objects_to_evict = 0;
|
||||||
*objects_to_evict = NULL;
|
*objects_to_evict = NULL;
|
||||||
}
|
}
|
||||||
|
|
||||||
void end_object_access(eviction_state *eviction_state,
|
void EvictionState_end_object_access(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict) {
|
ObjectID **objects_to_evict) {
|
||||||
/* Add the object to the LRU cache.*/
|
/* Add the object to the LRU cache.*/
|
||||||
add_object_to_lru_cache(eviction_state, obj_id);
|
add_object_to_lru_cache(eviction_state, obj_id);
|
||||||
*num_objects_to_evict = 0;
|
*num_objects_to_evict = 0;
|
||||||
|
|
|
@ -11,7 +11,7 @@
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/** Internal state of the eviction policy. */
|
/** Internal state of the eviction policy. */
|
||||||
typedef struct eviction_state eviction_state;
|
typedef struct EvictionState EvictionState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the eviction policy state.
|
* Initialize the eviction policy state.
|
||||||
|
@ -20,7 +20,7 @@ typedef struct eviction_state eviction_state;
|
||||||
* store.
|
* store.
|
||||||
* @return The internal state of the eviction policy.
|
* @return The internal state of the eviction policy.
|
||||||
*/
|
*/
|
||||||
eviction_state *make_eviction_state(void);
|
EvictionState *EvictionState_init(void);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Free the eviction policy state.
|
* Free the eviction policy state.
|
||||||
|
@ -28,7 +28,7 @@ eviction_state *make_eviction_state(void);
|
||||||
* @param state The state managed by the eviction policy.
|
* @param state The state managed by the eviction policy.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void free_eviction_state(eviction_state *state);
|
void EvictionState_free(EvictionState *state);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will be called whenever an object is first created in order to
|
* This method will be called whenever an object is first created in order to
|
||||||
|
@ -41,9 +41,9 @@ void free_eviction_state(eviction_state *state);
|
||||||
* @param obj_id The object ID of the object that was created.
|
* @param obj_id The object ID of the object that was created.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_created(eviction_state *eviction_state,
|
void EvictionState_object_created(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id);
|
ObjectID obj_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will be called when the Plasma store needs more space, perhaps to
|
* This method will be called when the Plasma store needs more space, perhaps to
|
||||||
|
@ -65,11 +65,11 @@ void object_created(eviction_state *eviction_state,
|
||||||
* the array will be NULL.
|
* the array will be NULL.
|
||||||
* @return True if enough space can be freed and false otherwise.
|
* @return True if enough space can be freed and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool require_space(eviction_state *eviction_state,
|
bool EvictionState_require_space(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
int64_t size,
|
int64_t size,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict);
|
ObjectID **objects_to_evict);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will be called whenever an unused object in the Plasma store
|
* This method will be called whenever an unused object in the Plasma store
|
||||||
|
@ -89,11 +89,11 @@ bool require_space(eviction_state *eviction_state,
|
||||||
* the array will be NULL.
|
* the array will be NULL.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void begin_object_access(eviction_state *eviction_state,
|
void EvictionState_begin_object_access(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict);
|
ObjectID **objects_to_evict);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will be called whenever an object in the Plasma store that was
|
* This method will be called whenever an object in the Plasma store that was
|
||||||
|
@ -113,11 +113,11 @@ void begin_object_access(eviction_state *eviction_state,
|
||||||
* the array will be NULL.
|
* the array will be NULL.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void end_object_access(eviction_state *eviction_state,
|
void EvictionState_end_object_access(EvictionState *eviction_state,
|
||||||
plasma_store_info *plasma_store_info,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t *num_objects_to_evict,
|
||||||
object_id **objects_to_evict);
|
ObjectID **objects_to_evict);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Choose some objects to evict from the Plasma store. When this method is
|
* Choose some objects to evict from the Plasma store. When this method is
|
||||||
|
@ -139,10 +139,11 @@ void end_object_access(eviction_state *eviction_state,
|
||||||
* the array will be NULL.
|
* the array will be NULL.
|
||||||
* @return The total number of bytes of space chosen to be evicted.
|
* @return The total number of bytes of space chosen to be evicted.
|
||||||
*/
|
*/
|
||||||
int64_t choose_objects_to_evict(eviction_state *eviction_state,
|
int64_t EvictionState_choose_objects_to_evict(
|
||||||
plasma_store_info *plasma_store_info,
|
EvictionState *eviction_state,
|
||||||
int64_t num_bytes_required,
|
PlasmaStoreInfo *plasma_store_info,
|
||||||
int64_t *num_objects_to_evict,
|
int64_t num_bytes_required,
|
||||||
object_id **objects_to_evict);
|
int64_t *num_objects_to_evict,
|
||||||
|
ObjectID **objects_to_evict);
|
||||||
|
|
||||||
#endif /* EVICTION_POLICY_H */
|
#endif /* EVICTION_POLICY_H */
|
||||||
|
|
|
@ -55,7 +55,7 @@ enum PlasmaError:int {
|
||||||
|
|
||||||
// Plasma store messages
|
// Plasma store messages
|
||||||
|
|
||||||
struct PlasmaObject {
|
struct PlasmaObjectSpec {
|
||||||
// Index of the memory segment (= memory mapped file) that
|
// Index of the memory segment (= memory mapped file) that
|
||||||
// this object is allocated in.
|
// this object is allocated in.
|
||||||
segment_index: int;
|
segment_index: int;
|
||||||
|
@ -84,7 +84,7 @@ table PlasmaCreateReply {
|
||||||
// ID of the object that was created.
|
// ID of the object that was created.
|
||||||
object_id: string;
|
object_id: string;
|
||||||
// The object that is returned with this reply.
|
// The object that is returned with this reply.
|
||||||
plasma_object: PlasmaObject;
|
plasma_object: PlasmaObjectSpec;
|
||||||
// Error that occurred for this call.
|
// Error that occurred for this call.
|
||||||
error: PlasmaError;
|
error: PlasmaError;
|
||||||
}
|
}
|
||||||
|
@ -117,7 +117,7 @@ table PlasmaGetReply {
|
||||||
// in the local Plasma store.
|
// in the local Plasma store.
|
||||||
object_ids: [string];
|
object_ids: [string];
|
||||||
// Plasma object information, in the same order as their IDs.
|
// Plasma object information, in the same order as their IDs.
|
||||||
plasma_objects: [PlasmaObject];
|
plasma_objects: [PlasmaObjectSpec];
|
||||||
// The number of elements in both object_ids and plasma_objects arrays must agree.
|
// The number of elements in both object_ids and plasma_objects arrays must agree.
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -7,17 +7,6 @@
|
||||||
|
|
||||||
#include "plasma_protocol.h"
|
#include "plasma_protocol.h"
|
||||||
|
|
||||||
bool plasma_object_ids_distinct(int num_object_ids, object_id object_ids[]) {
|
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
|
||||||
for (int j = 0; j < i; ++j) {
|
|
||||||
if (object_ids_equal(object_ids[i], object_ids[j])) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
void warn_if_sigpipe(int status, int client_sock) {
|
void warn_if_sigpipe(int status, int client_sock) {
|
||||||
if (status >= 0) {
|
if (status >= 0) {
|
||||||
return;
|
return;
|
||||||
|
|
|
@ -22,7 +22,7 @@
|
||||||
*/
|
*/
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the requested object. If ID_NIL request any object. */
|
/** The ID of the requested object. If ID_NIL request any object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Request associated to the object. It can take one of the following values:
|
/** Request associated to the object. It can take one of the following values:
|
||||||
* - PLASMA_QUERY_LOCAL: return if or when the object is available in the
|
* - PLASMA_QUERY_LOCAL: return if or when the object is available in the
|
||||||
* local Plasma Store.
|
* local Plasma Store.
|
||||||
|
@ -37,7 +37,7 @@ typedef struct {
|
||||||
* - PLASMA_CLIENT_IN_TRANSFER, if the object is currently being scheduled
|
* - PLASMA_CLIENT_IN_TRANSFER, if the object is currently being scheduled
|
||||||
* for being transferred or it is transferring. */
|
* for being transferred or it is transferring. */
|
||||||
int status;
|
int status;
|
||||||
} object_request;
|
} ObjectRequest;
|
||||||
|
|
||||||
/* Handle to access memory mapped file and map it into client address space. */
|
/* Handle to access memory mapped file and map it into client address space. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
|
@ -60,7 +60,7 @@ typedef struct {
|
||||||
int64_t data_size;
|
int64_t data_size;
|
||||||
/** The size in bytes of the metadata. */
|
/** The size in bytes of the metadata. */
|
||||||
int64_t metadata_size;
|
int64_t metadata_size;
|
||||||
} plasma_object;
|
} PlasmaObject;
|
||||||
|
|
||||||
typedef enum {
|
typedef enum {
|
||||||
/** Object was created but not sealed in the local Plasma Store. */
|
/** Object was created but not sealed in the local Plasma Store. */
|
||||||
|
@ -87,9 +87,9 @@ typedef enum {
|
||||||
* the eviction policy. */
|
* the eviction policy. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Object id of this object. */
|
/** Object id of this object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Object info like size, creation time and owner. */
|
/** Object info like size, creation time and owner. */
|
||||||
object_info info;
|
ObjectInfo info;
|
||||||
/** Memory mapped file containing the object. */
|
/** Memory mapped file containing the object. */
|
||||||
int fd;
|
int fd;
|
||||||
/** Size of the underlying map. */
|
/** Size of the underlying map. */
|
||||||
|
@ -115,25 +115,7 @@ typedef struct {
|
||||||
/** The amount of memory (in bytes) that we allow to be allocated in the
|
/** The amount of memory (in bytes) that we allow to be allocated in the
|
||||||
* store. */
|
* store. */
|
||||||
int64_t memory_capacity;
|
int64_t memory_capacity;
|
||||||
} plasma_store_info;
|
} PlasmaStoreInfo;
|
||||||
|
|
||||||
typedef struct {
|
|
||||||
/** The ID of the object. */
|
|
||||||
object_id obj_id;
|
|
||||||
/** The size of the object. */
|
|
||||||
int64_t object_size;
|
|
||||||
/** The digest of the object used, used to see if two objects are the same. */
|
|
||||||
unsigned char digest[DIGEST_SIZE];
|
|
||||||
} object_id_notification;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check if a collection of object IDs contains any duplicates.
|
|
||||||
*
|
|
||||||
* @param num_object_ids The number of object IDs.
|
|
||||||
* @param object_ids[] The list of object IDs to check.
|
|
||||||
* @return True if the object IDs are all distinct and false otherwise.
|
|
||||||
*/
|
|
||||||
bool plasma_object_ids_distinct(int num_object_ids, object_id object_ids[]);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Print a warning if the status is less than zero. This should be used to check
|
* Print a warning if the status is less than zero. This should be used to check
|
||||||
|
|
|
@ -52,14 +52,14 @@ typedef struct {
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the object. This is used as the key in the hash table. */
|
/** The ID of the object. This is used as the key in the hash table. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** A count of the number of times this client has called plasma_create or
|
/** A count of the number of times this client has called plasma_create or
|
||||||
* plasma_get on this object ID minus the number of calls to plasma_release.
|
* plasma_get on this object ID minus the number of calls to plasma_release.
|
||||||
* When this count reaches zero, we remove the entry from the objects_in_use
|
* When this count reaches zero, we remove the entry from the objects_in_use
|
||||||
* and decrement a count in the relevant client_mmap_table_entry. */
|
* and decrement a count in the relevant client_mmap_table_entry. */
|
||||||
int count;
|
int count;
|
||||||
/** Cached information to read the object. */
|
/** Cached information to read the object. */
|
||||||
plasma_object object;
|
PlasmaObject object;
|
||||||
/** A flag representing whether the object has been sealed. */
|
/** A flag representing whether the object has been sealed. */
|
||||||
bool is_sealed;
|
bool is_sealed;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
|
@ -78,7 +78,7 @@ typedef struct {
|
||||||
* is used to implement the delayed release mechanism. */
|
* is used to implement the delayed release mechanism. */
|
||||||
typedef struct pending_release {
|
typedef struct pending_release {
|
||||||
/** The object_id of the released object. */
|
/** The object_id of the released object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Needed for the doubly-linked list macros. */
|
/** Needed for the doubly-linked list macros. */
|
||||||
struct pending_release *prev;
|
struct pending_release *prev;
|
||||||
/** Needed for the doubly-linked list macros. */
|
/** Needed for the doubly-linked list macros. */
|
||||||
|
@ -87,7 +87,7 @@ typedef struct pending_release {
|
||||||
|
|
||||||
/** Information about a connection between a Plasma Client and Plasma Store.
|
/** Information about a connection between a Plasma Client and Plasma Store.
|
||||||
* This is used to avoid mapping the same files into memory multiple times. */
|
* This is used to avoid mapping the same files into memory multiple times. */
|
||||||
struct plasma_connection {
|
struct PlasmaConnection {
|
||||||
/** File descriptor of the Unix domain socket that connects to the store. */
|
/** File descriptor of the Unix domain socket that connects to the store. */
|
||||||
int store_conn;
|
int store_conn;
|
||||||
/** File descriptor of the Unix domain socket that connects to the manager. */
|
/** File descriptor of the Unix domain socket that connects to the manager. */
|
||||||
|
@ -130,7 +130,7 @@ struct plasma_connection {
|
||||||
/* If the file descriptor fd has been mmapped in this client process before,
|
/* If the file descriptor fd has been mmapped in this client process before,
|
||||||
* return the pointer that was returned by mmap, otherwise mmap it and store the
|
* return the pointer that was returned by mmap, otherwise mmap it and store the
|
||||||
* pointer in a hash table. */
|
* pointer in a hash table. */
|
||||||
uint8_t *lookup_or_mmap(plasma_connection *conn,
|
uint8_t *lookup_or_mmap(PlasmaConnection *conn,
|
||||||
int fd,
|
int fd,
|
||||||
int store_fd_val,
|
int store_fd_val,
|
||||||
int64_t map_size) {
|
int64_t map_size) {
|
||||||
|
@ -158,16 +158,16 @@ uint8_t *lookup_or_mmap(plasma_connection *conn,
|
||||||
|
|
||||||
/* Get a pointer to a file that we know has been memory mapped in this client
|
/* Get a pointer to a file that we know has been memory mapped in this client
|
||||||
* process before. */
|
* process before. */
|
||||||
uint8_t *lookup_mmapped_file(plasma_connection *conn, int store_fd_val) {
|
uint8_t *lookup_mmapped_file(PlasmaConnection *conn, int store_fd_val) {
|
||||||
client_mmap_table_entry *entry;
|
client_mmap_table_entry *entry;
|
||||||
HASH_FIND_INT(conn->mmap_table, &store_fd_val, entry);
|
HASH_FIND_INT(conn->mmap_table, &store_fd_val, entry);
|
||||||
CHECK(entry);
|
CHECK(entry);
|
||||||
return entry->pointer;
|
return entry->pointer;
|
||||||
}
|
}
|
||||||
|
|
||||||
void increment_object_count(plasma_connection *conn,
|
void increment_object_count(PlasmaConnection *conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
plasma_object *object,
|
PlasmaObject *object,
|
||||||
bool is_sealed) {
|
bool is_sealed) {
|
||||||
/* Increment the count of the object to track the fact that it is being used.
|
/* Increment the count of the object to track the fact that it is being used.
|
||||||
* The corresponding decrement should happen in plasma_release. */
|
* The corresponding decrement should happen in plasma_release. */
|
||||||
|
@ -204,8 +204,8 @@ void increment_object_count(plasma_connection *conn,
|
||||||
object_entry->count += 1;
|
object_entry->count += 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
int plasma_create(plasma_connection *conn,
|
int plasma_create(PlasmaConnection *conn,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
uint8_t *metadata,
|
uint8_t *metadata,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
|
@ -218,8 +218,8 @@ int plasma_create(plasma_connection *conn,
|
||||||
uint8_t *reply_data =
|
uint8_t *reply_data =
|
||||||
plasma_receive(conn->store_conn, MessageType_PlasmaCreateReply);
|
plasma_receive(conn->store_conn, MessageType_PlasmaCreateReply);
|
||||||
int error;
|
int error;
|
||||||
object_id id;
|
ObjectID id;
|
||||||
plasma_object object;
|
PlasmaObject object;
|
||||||
plasma_read_CreateReply(reply_data, &id, &object, &error);
|
plasma_read_CreateReply(reply_data, &id, &object, &error);
|
||||||
free(reply_data);
|
free(reply_data);
|
||||||
if (error != PlasmaError_OK) {
|
if (error != PlasmaError_OK) {
|
||||||
|
@ -258,11 +258,11 @@ int plasma_create(plasma_connection *conn,
|
||||||
return PlasmaError_OK;
|
return PlasmaError_OK;
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_get(plasma_connection *conn,
|
void plasma_get(PlasmaConnection *conn,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects,
|
int64_t num_objects,
|
||||||
int64_t timeout_ms,
|
int64_t timeout_ms,
|
||||||
object_buffer object_buffers[]) {
|
ObjectBuffer object_buffers[]) {
|
||||||
/* Fill out the info for the objects that are already in use locally. */
|
/* Fill out the info for the objects that are already in use locally. */
|
||||||
bool all_present = true;
|
bool all_present = true;
|
||||||
for (int i = 0; i < num_objects; ++i) {
|
for (int i = 0; i < num_objects; ++i) {
|
||||||
|
@ -277,8 +277,8 @@ void plasma_get(plasma_connection *conn,
|
||||||
object_buffers[i].data_size = -1;
|
object_buffers[i].data_size = -1;
|
||||||
} else {
|
} else {
|
||||||
/* */
|
/* */
|
||||||
plasma_object object_data;
|
PlasmaObject object_data;
|
||||||
plasma_object *object;
|
PlasmaObject *object;
|
||||||
/* NOTE: If the object is still unsealed, we will deadlock, since we must
|
/* NOTE: If the object is still unsealed, we will deadlock, since we must
|
||||||
* have been the one who created it. */
|
* have been the one who created it. */
|
||||||
CHECKM(object_entry->is_sealed,
|
CHECKM(object_entry->is_sealed,
|
||||||
|
@ -307,14 +307,14 @@ void plasma_get(plasma_connection *conn,
|
||||||
num_objects, timeout_ms) >= 0);
|
num_objects, timeout_ms) >= 0);
|
||||||
uint8_t *reply_data =
|
uint8_t *reply_data =
|
||||||
plasma_receive(conn->store_conn, MessageType_PlasmaGetReply);
|
plasma_receive(conn->store_conn, MessageType_PlasmaGetReply);
|
||||||
object_id *received_obj_ids = malloc(num_objects * sizeof(object_id));
|
ObjectID *received_obj_ids = malloc(num_objects * sizeof(ObjectID));
|
||||||
plasma_object *object_data = malloc(num_objects * sizeof(plasma_object));
|
PlasmaObject *object_data = malloc(num_objects * sizeof(PlasmaObject));
|
||||||
plasma_object *object;
|
PlasmaObject *object;
|
||||||
plasma_read_GetReply(reply_data, received_obj_ids, object_data, num_objects);
|
plasma_read_GetReply(reply_data, received_obj_ids, object_data, num_objects);
|
||||||
free(reply_data);
|
free(reply_data);
|
||||||
|
|
||||||
for (int i = 0; i < num_objects; ++i) {
|
for (int i = 0; i < num_objects; ++i) {
|
||||||
DCHECK(object_ids_equal(received_obj_ids[i], object_ids[i]));
|
DCHECK(ObjectID_equal(received_obj_ids[i], object_ids[i]));
|
||||||
object = &object_data[i];
|
object = &object_data[i];
|
||||||
if (object_buffers[i].data_size != -1) {
|
if (object_buffers[i].data_size != -1) {
|
||||||
/* If the object was already in use by the client, then the store should
|
/* If the object was already in use by the client, then the store should
|
||||||
|
@ -371,7 +371,7 @@ void plasma_get(plasma_connection *conn,
|
||||||
* @param conn The plasma connection.
|
* @param conn The plasma connection.
|
||||||
* @param object_id The object ID to attempt to release.
|
* @param object_id The object ID to attempt to release.
|
||||||
*/
|
*/
|
||||||
void plasma_perform_release(plasma_connection *conn, object_id object_id) {
|
void plasma_perform_release(PlasmaConnection *conn, ObjectID object_id) {
|
||||||
/* Decrement the count of the number of instances of this object that are
|
/* Decrement the count of the number of instances of this object that are
|
||||||
* being used by this client. The corresponding increment should have happened
|
* being used by this client. The corresponding increment should have happened
|
||||||
* in plasma_get. */
|
* in plasma_get. */
|
||||||
|
@ -412,7 +412,7 @@ void plasma_perform_release(plasma_connection *conn, object_id object_id) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_release(plasma_connection *conn, object_id obj_id) {
|
void plasma_release(PlasmaConnection *conn, ObjectID obj_id) {
|
||||||
/* Add the new object to the release history. The corresponding call to free
|
/* Add the new object to the release history. The corresponding call to free
|
||||||
* will occur in plasma_perform_release or in plasma_disconnect. */
|
* will occur in plasma_perform_release or in plasma_disconnect. */
|
||||||
pending_release *pending_release_entry = malloc(sizeof(pending_release));
|
pending_release *pending_release_entry = malloc(sizeof(pending_release));
|
||||||
|
@ -443,9 +443,7 @@ void plasma_release(plasma_connection *conn, object_id obj_id) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* This method is used to query whether the plasma store contains an object. */
|
/* This method is used to query whether the plasma store contains an object. */
|
||||||
void plasma_contains(plasma_connection *conn,
|
void plasma_contains(PlasmaConnection *conn, ObjectID obj_id, int *has_object) {
|
||||||
object_id obj_id,
|
|
||||||
int *has_object) {
|
|
||||||
/* Check if we already have a reference to the object. */
|
/* Check if we already have a reference to the object. */
|
||||||
object_in_use_entry *object_entry;
|
object_in_use_entry *object_entry;
|
||||||
HASH_FIND(hh, conn->objects_in_use, &obj_id, sizeof(obj_id), object_entry);
|
HASH_FIND(hh, conn->objects_in_use, &obj_id, sizeof(obj_id), object_entry);
|
||||||
|
@ -457,19 +455,19 @@ void plasma_contains(plasma_connection *conn,
|
||||||
plasma_send_ContainsRequest(conn->store_conn, conn->builder, obj_id);
|
plasma_send_ContainsRequest(conn->store_conn, conn->builder, obj_id);
|
||||||
uint8_t *reply_data =
|
uint8_t *reply_data =
|
||||||
plasma_receive(conn->store_conn, MessageType_PlasmaContainsReply);
|
plasma_receive(conn->store_conn, MessageType_PlasmaContainsReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
plasma_read_ContainsReply(reply_data, &object_id2, has_object);
|
plasma_read_ContainsReply(reply_data, &object_id2, has_object);
|
||||||
free(reply_data);
|
free(reply_data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool plasma_compute_object_hash(plasma_connection *conn,
|
bool plasma_compute_object_hash(PlasmaConnection *conn,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
unsigned char *digest) {
|
unsigned char *digest) {
|
||||||
/* Get the plasma object data. We pass in a timeout of 0 to indicate that
|
/* Get the plasma object data. We pass in a timeout of 0 to indicate that
|
||||||
* the operation should timeout immediately. */
|
* the operation should timeout immediately. */
|
||||||
object_buffer obj_buffer;
|
ObjectBuffer obj_buffer;
|
||||||
object_id obj_id_array[1] = {obj_id};
|
ObjectID obj_id_array[1] = {obj_id};
|
||||||
plasma_get(conn, obj_id_array, 1, 0, &obj_buffer);
|
plasma_get(conn, obj_id_array, 1, 0, &obj_buffer);
|
||||||
/* If the object was not retrieved, return false. */
|
/* If the object was not retrieved, return false. */
|
||||||
if (obj_buffer.data_size == -1) {
|
if (obj_buffer.data_size == -1) {
|
||||||
|
@ -491,7 +489,7 @@ bool plasma_compute_object_hash(plasma_connection *conn,
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_seal(plasma_connection *conn, object_id object_id) {
|
void plasma_seal(PlasmaConnection *conn, ObjectID object_id) {
|
||||||
/* Make sure this client has a reference to the object before sending the
|
/* Make sure this client has a reference to the object before sending the
|
||||||
* request to Plasma. */
|
* request to Plasma. */
|
||||||
object_in_use_entry *object_entry;
|
object_in_use_entry *object_entry;
|
||||||
|
@ -514,12 +512,12 @@ void plasma_seal(plasma_connection *conn, object_id object_id) {
|
||||||
plasma_release(conn, object_id);
|
plasma_release(conn, object_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_delete(plasma_connection *conn, object_id object_id) {
|
void plasma_delete(PlasmaConnection *conn, ObjectID object_id) {
|
||||||
/* TODO(rkn): In the future, we can use this method to give hints to the
|
/* TODO(rkn): In the future, we can use this method to give hints to the
|
||||||
* eviction policy about when an object will no longer be needed. */
|
* eviction policy about when an object will no longer be needed. */
|
||||||
}
|
}
|
||||||
|
|
||||||
int64_t plasma_evict(plasma_connection *conn, int64_t num_bytes) {
|
int64_t plasma_evict(PlasmaConnection *conn, int64_t num_bytes) {
|
||||||
/* Send a request to the store to evict objects. */
|
/* Send a request to the store to evict objects. */
|
||||||
CHECK(plasma_send_EvictRequest(conn->store_conn, conn->builder, num_bytes) >=
|
CHECK(plasma_send_EvictRequest(conn->store_conn, conn->builder, num_bytes) >=
|
||||||
0);
|
0);
|
||||||
|
@ -534,7 +532,7 @@ int64_t plasma_evict(plasma_connection *conn, int64_t num_bytes) {
|
||||||
return num_bytes_evicted;
|
return num_bytes_evicted;
|
||||||
}
|
}
|
||||||
|
|
||||||
int plasma_subscribe(plasma_connection *conn) {
|
int plasma_subscribe(PlasmaConnection *conn) {
|
||||||
int fd[2];
|
int fd[2];
|
||||||
/* TODO: Just create 1 socket, bind it to port 0 to find a free port, and
|
/* TODO: Just create 1 socket, bind it to port 0 to find a free port, and
|
||||||
* send the port number instead, and let the client connect. */
|
* send the port number instead, and let the client connect. */
|
||||||
|
@ -555,11 +553,11 @@ int plasma_subscribe(plasma_connection *conn) {
|
||||||
return fd[0];
|
return fd[0];
|
||||||
}
|
}
|
||||||
|
|
||||||
plasma_connection *plasma_connect(const char *store_socket_name,
|
PlasmaConnection *plasma_connect(const char *store_socket_name,
|
||||||
const char *manager_socket_name,
|
const char *manager_socket_name,
|
||||||
int release_delay) {
|
int release_delay) {
|
||||||
/* Initialize the store connection struct */
|
/* Initialize the store connection struct */
|
||||||
plasma_connection *result = malloc(sizeof(plasma_connection));
|
PlasmaConnection *result = malloc(sizeof(PlasmaConnection));
|
||||||
result->store_conn = connect_ipc_sock_retry(store_socket_name, -1, -1);
|
result->store_conn = connect_ipc_sock_retry(store_socket_name, -1, -1);
|
||||||
if (manager_socket_name != NULL) {
|
if (manager_socket_name != NULL) {
|
||||||
result->manager_conn = connect_ipc_sock_retry(manager_socket_name, -1, -1);
|
result->manager_conn = connect_ipc_sock_retry(manager_socket_name, -1, -1);
|
||||||
|
@ -584,7 +582,7 @@ plasma_connection *plasma_connect(const char *store_socket_name,
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_disconnect(plasma_connection *conn) {
|
void plasma_disconnect(PlasmaConnection *conn) {
|
||||||
/* Perform the pending release calls to flush out the queue so that the counts
|
/* Perform the pending release calls to flush out the queue so that the counts
|
||||||
* in the objects_in_use table are accurate. */
|
* in the objects_in_use table are accurate. */
|
||||||
pending_release *element, *temp;
|
pending_release *element, *temp;
|
||||||
|
@ -596,7 +594,7 @@ void plasma_disconnect(plasma_connection *conn) {
|
||||||
/* Loop over the objects in use table and release all remaining objects. */
|
/* Loop over the objects in use table and release all remaining objects. */
|
||||||
object_in_use_entry *current_entry, *temp_entry;
|
object_in_use_entry *current_entry, *temp_entry;
|
||||||
HASH_ITER(hh, conn->objects_in_use, current_entry, temp_entry) {
|
HASH_ITER(hh, conn->objects_in_use, current_entry, temp_entry) {
|
||||||
object_id object_id_to_release = current_entry->object_id;
|
ObjectID object_id_to_release = current_entry->object_id;
|
||||||
int count = current_entry->count;
|
int count = current_entry->count;
|
||||||
for (int i = 0; i < count; ++i) {
|
for (int i = 0; i < count; ++i) {
|
||||||
plasma_perform_release(conn, object_id_to_release);
|
plasma_perform_release(conn, object_id_to_release);
|
||||||
|
@ -613,34 +611,34 @@ void plasma_disconnect(plasma_connection *conn) {
|
||||||
free(conn);
|
free(conn);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool plasma_manager_is_connected(plasma_connection *conn) {
|
bool plasma_manager_is_connected(PlasmaConnection *conn) {
|
||||||
return conn->manager_conn >= 0;
|
return conn->manager_conn >= 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
#define h_addr h_addr_list[0]
|
#define h_addr h_addr_list[0]
|
||||||
|
|
||||||
void plasma_transfer(plasma_connection *conn,
|
void plasma_transfer(PlasmaConnection *conn,
|
||||||
const char *address,
|
const char *address,
|
||||||
int port,
|
int port,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
CHECK(plasma_send_DataRequest(conn->manager_conn, conn->builder, object_id,
|
CHECK(plasma_send_DataRequest(conn->manager_conn, conn->builder, object_id,
|
||||||
address, port) >= 0);
|
address, port) >= 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_fetch(plasma_connection *conn,
|
void plasma_fetch(PlasmaConnection *conn,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[]) {
|
ObjectID object_ids[]) {
|
||||||
CHECK(conn != NULL);
|
CHECK(conn != NULL);
|
||||||
CHECK(conn->manager_conn >= 0);
|
CHECK(conn->manager_conn >= 0);
|
||||||
CHECK(plasma_send_FetchRequest(conn->manager_conn, conn->builder, object_ids,
|
CHECK(plasma_send_FetchRequest(conn->manager_conn, conn->builder, object_ids,
|
||||||
num_object_ids) >= 0);
|
num_object_ids) >= 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
int get_manager_fd(plasma_connection *conn) {
|
int get_manager_fd(PlasmaConnection *conn) {
|
||||||
return conn->manager_conn;
|
return conn->manager_conn;
|
||||||
}
|
}
|
||||||
|
|
||||||
int plasma_status(plasma_connection *conn, object_id object_id) {
|
int plasma_status(PlasmaConnection *conn, ObjectID object_id) {
|
||||||
CHECK(conn != NULL);
|
CHECK(conn != NULL);
|
||||||
CHECK(conn->manager_conn >= 0);
|
CHECK(conn->manager_conn >= 0);
|
||||||
|
|
||||||
|
@ -653,9 +651,9 @@ int plasma_status(plasma_connection *conn, object_id object_id) {
|
||||||
return object_status;
|
return object_status;
|
||||||
}
|
}
|
||||||
|
|
||||||
int plasma_wait(plasma_connection *conn,
|
int plasma_wait(PlasmaConnection *conn,
|
||||||
int num_object_requests,
|
int num_object_requests,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_ready_objects,
|
int num_ready_objects,
|
||||||
uint64_t timeout_ms) {
|
uint64_t timeout_ms) {
|
||||||
CHECK(conn != NULL);
|
CHECK(conn != NULL);
|
||||||
|
|
|
@ -10,7 +10,7 @@
|
||||||
/* Use 100MB as an overestimate of the L3 cache size. */
|
/* Use 100MB as an overestimate of the L3 cache size. */
|
||||||
#define L3_CACHE_SIZE_BYTES 100000000
|
#define L3_CACHE_SIZE_BYTES 100000000
|
||||||
|
|
||||||
typedef struct plasma_connection plasma_connection;
|
typedef struct PlasmaConnection PlasmaConnection;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Try to connect to the socket several times. If unsuccessful, fail.
|
* Try to connect to the socket several times. If unsuccessful, fail.
|
||||||
|
@ -35,9 +35,9 @@ int socket_connect_retry(const char *socket_name,
|
||||||
* function will not connect to a manager.
|
* function will not connect to a manager.
|
||||||
* @return The object containing the connection state.
|
* @return The object containing the connection state.
|
||||||
*/
|
*/
|
||||||
plasma_connection *plasma_connect(const char *store_socket_name,
|
PlasmaConnection *plasma_connect(const char *store_socket_name,
|
||||||
const char *manager_socket_name,
|
const char *manager_socket_name,
|
||||||
int release_delay);
|
int release_delay);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Disconnect from the local plasma instance, including the local store and
|
* Disconnect from the local plasma instance, including the local store and
|
||||||
|
@ -46,7 +46,7 @@ plasma_connection *plasma_connect(const char *store_socket_name,
|
||||||
* @param conn The connection to the local plasma store and plasma manager.
|
* @param conn The connection to the local plasma store and plasma manager.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_disconnect(plasma_connection *conn);
|
void plasma_disconnect(PlasmaConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return true if the plasma manager is connected.
|
* Return true if the plasma manager is connected.
|
||||||
|
@ -54,7 +54,7 @@ void plasma_disconnect(plasma_connection *conn);
|
||||||
* @param conn The connection to the local plasma store and plasma manager.
|
* @param conn The connection to the local plasma store and plasma manager.
|
||||||
* @return True if the plasma manager is connected and false otherwise.
|
* @return True if the plasma manager is connected and false otherwise.
|
||||||
*/
|
*/
|
||||||
bool plasma_manager_is_connected(plasma_connection *conn);
|
bool plasma_manager_is_connected(PlasmaConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Try to connect to a possibly remote Plasma Manager.
|
* Try to connect to a possibly remote Plasma Manager.
|
||||||
|
@ -89,8 +89,8 @@ int plasma_manager_connect(const char *addr, int port);
|
||||||
* create the object. In this case, the client should not call
|
* create the object. In this case, the client should not call
|
||||||
* plasma_release.
|
* plasma_release.
|
||||||
*/
|
*/
|
||||||
int plasma_create(plasma_connection *conn,
|
int plasma_create(PlasmaConnection *conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t size,
|
int64_t size,
|
||||||
uint8_t *metadata,
|
uint8_t *metadata,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
|
@ -108,7 +108,7 @@ typedef struct {
|
||||||
int64_t metadata_size;
|
int64_t metadata_size;
|
||||||
/** The address of the metadata. */
|
/** The address of the metadata. */
|
||||||
uint8_t *metadata;
|
uint8_t *metadata;
|
||||||
} object_buffer;
|
} ObjectBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get some objects from the Plasma Store. This function will block until the
|
* Get some objects from the Plasma Store. This function will block until the
|
||||||
|
@ -125,11 +125,11 @@ typedef struct {
|
||||||
* size field is -1, then the object was not retrieved.
|
* size field is -1, then the object was not retrieved.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_get(plasma_connection *conn,
|
void plasma_get(PlasmaConnection *conn,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects,
|
int64_t num_objects,
|
||||||
int64_t timeout_ms,
|
int64_t timeout_ms,
|
||||||
object_buffer object_buffers[]);
|
ObjectBuffer object_buffers[]);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tell Plasma that the client no longer needs the object. This should be called
|
* Tell Plasma that the client no longer needs the object. This should be called
|
||||||
|
@ -141,7 +141,7 @@ void plasma_get(plasma_connection *conn,
|
||||||
* @param object_id The ID of the object that is no longer needed.
|
* @param object_id The ID of the object that is no longer needed.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_release(plasma_connection *conn, object_id object_id);
|
void plasma_release(PlasmaConnection *conn, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if the object store contains a particular object and the object has
|
* Check if the object store contains a particular object and the object has
|
||||||
|
@ -155,8 +155,8 @@ void plasma_release(plasma_connection *conn, object_id object_id);
|
||||||
* present and 0 if it is not present.
|
* present and 0 if it is not present.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_contains(plasma_connection *conn,
|
void plasma_contains(PlasmaConnection *conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int *has_object);
|
int *has_object);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -168,8 +168,8 @@ void plasma_contains(plasma_connection *conn,
|
||||||
* The pointer must have at least DIGEST_SIZE bytes allocated.
|
* The pointer must have at least DIGEST_SIZE bytes allocated.
|
||||||
* @return A boolean representing whether the hash operation succeeded.
|
* @return A boolean representing whether the hash operation succeeded.
|
||||||
*/
|
*/
|
||||||
bool plasma_compute_object_hash(plasma_connection *conn,
|
bool plasma_compute_object_hash(PlasmaConnection *conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
unsigned char *digest);
|
unsigned char *digest);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -180,7 +180,7 @@ bool plasma_compute_object_hash(plasma_connection *conn,
|
||||||
* @param object_id The ID of the object to seal.
|
* @param object_id The ID of the object to seal.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_seal(plasma_connection *conn, object_id object_id);
|
void plasma_seal(PlasmaConnection *conn, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete an object from the object store. This currently assumes that the
|
* Delete an object from the object store. This currently assumes that the
|
||||||
|
@ -193,7 +193,7 @@ void plasma_seal(plasma_connection *conn, object_id object_id);
|
||||||
* @param object_id The ID of the object to delete.
|
* @param object_id The ID of the object to delete.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_delete(plasma_connection *conn, object_id object_id);
|
void plasma_delete(PlasmaConnection *conn, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete objects until we have freed up num_bytes bytes or there are no more
|
* Delete objects until we have freed up num_bytes bytes or there are no more
|
||||||
|
@ -203,7 +203,7 @@ void plasma_delete(plasma_connection *conn, object_id object_id);
|
||||||
* @param num_bytes The number of bytes to try to free up.
|
* @param num_bytes The number of bytes to try to free up.
|
||||||
* @return The total number of bytes of space retrieved.
|
* @return The total number of bytes of space retrieved.
|
||||||
*/
|
*/
|
||||||
int64_t plasma_evict(plasma_connection *conn, int64_t num_bytes);
|
int64_t plasma_evict(PlasmaConnection *conn, int64_t num_bytes);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Attempt to initiate the transfer of some objects from remote Plasma Stores.
|
* Attempt to initiate the transfer of some objects from remote Plasma Stores.
|
||||||
|
@ -226,9 +226,9 @@ int64_t plasma_evict(plasma_connection *conn, int64_t num_bytes);
|
||||||
* @param object_ids The IDs of the objects that fetch is being called on.
|
* @param object_ids The IDs of the objects that fetch is being called on.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_fetch(plasma_connection *conn,
|
void plasma_fetch(PlasmaConnection *conn,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[]);
|
ObjectID object_ids[]);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transfer local object to a different plasma manager.
|
* Transfer local object to a different plasma manager.
|
||||||
|
@ -240,10 +240,10 @@ void plasma_fetch(plasma_connection *conn,
|
||||||
*
|
*
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void plasma_transfer(plasma_connection *conn,
|
void plasma_transfer(PlasmaConnection *conn,
|
||||||
const char *addr,
|
const char *addr,
|
||||||
int port,
|
int port,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subscribe to notifications when objects are sealed in the object store.
|
* Subscribe to notifications when objects are sealed in the object store.
|
||||||
|
@ -254,7 +254,7 @@ void plasma_transfer(plasma_connection *conn,
|
||||||
* @return The file descriptor that the client should use to read notifications
|
* @return The file descriptor that the client should use to read notifications
|
||||||
from the object store about sealed objects.
|
from the object store about sealed objects.
|
||||||
*/
|
*/
|
||||||
int plasma_subscribe(plasma_connection *conn);
|
int plasma_subscribe(PlasmaConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the file descriptor for the socket connection to the plasma manager.
|
* Get the file descriptor for the socket connection to the plasma manager.
|
||||||
|
@ -263,7 +263,7 @@ int plasma_subscribe(plasma_connection *conn);
|
||||||
* @return The file descriptor for the manager connection. If there is no
|
* @return The file descriptor for the manager connection. If there is no
|
||||||
* connection to the manager, this is -1.
|
* connection to the manager, this is -1.
|
||||||
*/
|
*/
|
||||||
int get_manager_fd(plasma_connection *conn);
|
int get_manager_fd(PlasmaConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the status of a given object. This method may query the object table.
|
* Return the status of a given object. This method may query the object table.
|
||||||
|
@ -281,7 +281,7 @@ int get_manager_fd(plasma_connection *conn);
|
||||||
* - PLASMA_CLIENT_DOES_NOT_EXIST, if the object doesn’t exist in the
|
* - PLASMA_CLIENT_DOES_NOT_EXIST, if the object doesn’t exist in the
|
||||||
* system.
|
* system.
|
||||||
*/
|
*/
|
||||||
int plasma_status(plasma_connection *conn, object_id object_id);
|
int plasma_status(PlasmaConnection *conn, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the information associated to a given object.
|
* Return the information associated to a given object.
|
||||||
|
@ -293,9 +293,9 @@ int plasma_status(plasma_connection *conn, object_id object_id);
|
||||||
* PLASMA_CLIENT_NOT_LOCAL, if not. In this case, the caller needs to
|
* PLASMA_CLIENT_NOT_LOCAL, if not. In this case, the caller needs to
|
||||||
* ignore data, metadata_size, and metadata fields.
|
* ignore data, metadata_size, and metadata fields.
|
||||||
*/
|
*/
|
||||||
int plasma_info(plasma_connection *conn,
|
int plasma_info(PlasmaConnection *conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
object_info *object_info);
|
ObjectInfo *object_info);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wait for (1) a specified number of objects to be available (sealed) in the
|
* Wait for (1) a specified number of objects to be available (sealed) in the
|
||||||
|
@ -326,9 +326,9 @@ int plasma_info(plasma_connection *conn,
|
||||||
* returned number is less than min_num_ready_objects this means that
|
* returned number is less than min_num_ready_objects this means that
|
||||||
* timeout expired.
|
* timeout expired.
|
||||||
*/
|
*/
|
||||||
int plasma_wait(plasma_connection *conn,
|
int plasma_wait(PlasmaConnection *conn,
|
||||||
int num_object_requests,
|
int num_object_requests,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_ready_objects,
|
int num_ready_objects,
|
||||||
uint64_t timeout_ms);
|
uint64_t timeout_ms);
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,7 @@ PyObject *PyPlasma_connect(PyObject *self, PyObject *args) {
|
||||||
&release_delay)) {
|
&release_delay)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
if (strlen(manager_socket_name) == 0) {
|
if (strlen(manager_socket_name) == 0) {
|
||||||
conn = plasma_connect(store_socket_name, NULL, release_delay);
|
conn = plasma_connect(store_socket_name, NULL, release_delay);
|
||||||
} else {
|
} else {
|
||||||
|
@ -33,7 +33,7 @@ PyObject *PyPlasma_connect(PyObject *self, PyObject *args) {
|
||||||
|
|
||||||
PyObject *PyPlasma_disconnect(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_disconnect(PyObject *self, PyObject *args) {
|
||||||
PyObject *conn_capsule;
|
PyObject *conn_capsule;
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
if (!PyArg_ParseTuple(args, "O", &conn_capsule)) {
|
if (!PyArg_ParseTuple(args, "O", &conn_capsule)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
|
@ -48,8 +48,8 @@ PyObject *PyPlasma_disconnect(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_create(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_create(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
long long size;
|
long long size;
|
||||||
PyObject *metadata;
|
PyObject *metadata;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&LO", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&LO", PyObjectToPlasmaConnection, &conn,
|
||||||
|
@ -86,8 +86,8 @@ PyObject *PyPlasma_create(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_hash(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_hash(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
||||||
PyStringToUniqueID, &object_id)) {
|
PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
|
@ -104,8 +104,8 @@ PyObject *PyPlasma_hash(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_seal(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_seal(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
||||||
PyStringToUniqueID, &object_id)) {
|
PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
|
@ -115,8 +115,8 @@ PyObject *PyPlasma_seal(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_release(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_release(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
||||||
PyStringToUniqueID, &object_id)) {
|
PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
|
@ -126,7 +126,7 @@ PyObject *PyPlasma_release(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_get(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_get(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
PyObject *object_id_list;
|
PyObject *object_id_list;
|
||||||
long long timeout_ms;
|
long long timeout_ms;
|
||||||
if (!PyArg_ParseTuple(args, "O&OL", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&OL", PyObjectToPlasmaConnection, &conn,
|
||||||
|
@ -135,8 +135,8 @@ PyObject *PyPlasma_get(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
Py_ssize_t num_object_ids = PyList_Size(object_id_list);
|
Py_ssize_t num_object_ids = PyList_Size(object_id_list);
|
||||||
object_id object_ids[num_object_ids];
|
ObjectID object_ids[num_object_ids];
|
||||||
object_buffer object_buffers[num_object_ids];
|
ObjectBuffer object_buffers[num_object_ids];
|
||||||
|
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
for (int i = 0; i < num_object_ids; ++i) {
|
||||||
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
||||||
|
@ -180,8 +180,8 @@ PyObject *PyPlasma_get(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_contains(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_contains(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
||||||
PyStringToUniqueID, &object_id)) {
|
PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
|
@ -196,7 +196,7 @@ PyObject *PyPlasma_contains(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_fetch(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_fetch(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
PyObject *object_id_list;
|
PyObject *object_id_list;
|
||||||
if (!PyArg_ParseTuple(args, "O&O", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O", PyObjectToPlasmaConnection, &conn,
|
||||||
&object_id_list)) {
|
&object_id_list)) {
|
||||||
|
@ -207,7 +207,7 @@ PyObject *PyPlasma_fetch(PyObject *self, PyObject *args) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
Py_ssize_t n = PyList_Size(object_id_list);
|
Py_ssize_t n = PyList_Size(object_id_list);
|
||||||
object_id *object_ids = malloc(sizeof(object_id) * n);
|
ObjectID *object_ids = malloc(sizeof(ObjectID) * n);
|
||||||
for (int i = 0; i < n; ++i) {
|
for (int i = 0; i < n; ++i) {
|
||||||
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
|
||||||
}
|
}
|
||||||
|
@ -217,7 +217,7 @@ PyObject *PyPlasma_fetch(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_wait(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_wait(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
PyObject *object_id_list;
|
PyObject *object_id_list;
|
||||||
long long timeout;
|
long long timeout;
|
||||||
int num_returns;
|
int num_returns;
|
||||||
|
@ -249,7 +249,7 @@ PyObject *PyPlasma_wait(PyObject *self, PyObject *args) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
|
|
||||||
object_request *object_requests = malloc(sizeof(object_request) * n);
|
ObjectRequest *object_requests = malloc(sizeof(ObjectRequest) * n);
|
||||||
for (int i = 0; i < n; ++i) {
|
for (int i = 0; i < n; ++i) {
|
||||||
CHECK(PyStringToUniqueID(PyList_GetItem(object_id_list, i),
|
CHECK(PyStringToUniqueID(PyList_GetItem(object_id_list, i),
|
||||||
&object_requests[i].object_id) == 1);
|
&object_requests[i].object_id) == 1);
|
||||||
|
@ -292,7 +292,7 @@ PyObject *PyPlasma_wait(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_evict(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_evict(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
long long num_bytes;
|
long long num_bytes;
|
||||||
if (!PyArg_ParseTuple(args, "O&L", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&L", PyObjectToPlasmaConnection, &conn,
|
||||||
&num_bytes)) {
|
&num_bytes)) {
|
||||||
|
@ -303,8 +303,8 @@ PyObject *PyPlasma_evict(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_delete(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_delete(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&", PyObjectToPlasmaConnection, &conn,
|
||||||
PyStringToUniqueID, &object_id)) {
|
PyStringToUniqueID, &object_id)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
|
@ -314,8 +314,8 @@ PyObject *PyPlasma_delete(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_transfer(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_transfer(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
const char *addr;
|
const char *addr;
|
||||||
int port;
|
int port;
|
||||||
if (!PyArg_ParseTuple(args, "O&O&si", PyObjectToPlasmaConnection, &conn,
|
if (!PyArg_ParseTuple(args, "O&O&si", PyObjectToPlasmaConnection, &conn,
|
||||||
|
@ -333,7 +333,7 @@ PyObject *PyPlasma_transfer(PyObject *self, PyObject *args) {
|
||||||
}
|
}
|
||||||
|
|
||||||
PyObject *PyPlasma_subscribe(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_subscribe(PyObject *self, PyObject *args) {
|
||||||
plasma_connection *conn;
|
PlasmaConnection *conn;
|
||||||
if (!PyArg_ParseTuple(args, "O&", PyObjectToPlasmaConnection, &conn)) {
|
if (!PyArg_ParseTuple(args, "O&", PyObjectToPlasmaConnection, &conn)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
|
@ -344,13 +344,13 @@ PyObject *PyPlasma_subscribe(PyObject *self, PyObject *args) {
|
||||||
|
|
||||||
PyObject *PyPlasma_receive_notification(PyObject *self, PyObject *args) {
|
PyObject *PyPlasma_receive_notification(PyObject *self, PyObject *args) {
|
||||||
int plasma_sock;
|
int plasma_sock;
|
||||||
object_info object_info;
|
ObjectInfo object_info;
|
||||||
|
|
||||||
if (!PyArg_ParseTuple(args, "i", &plasma_sock)) {
|
if (!PyArg_ParseTuple(args, "i", &plasma_sock)) {
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
/* Receive object notification from the plasma connection socket. If the
|
/* Receive object notification from the plasma connection socket. If the
|
||||||
* object was added, return a tuple of its fields: object_id, data_size,
|
* object was added, return a tuple of its fields: ObjectID, data_size,
|
||||||
* metadata_size. If the object was deleted, data_size and metadata_size will
|
* metadata_size. If the object was deleted, data_size and metadata_size will
|
||||||
* be set to -1. */
|
* be set to -1. */
|
||||||
int nbytes =
|
int nbytes =
|
||||||
|
|
|
@ -2,9 +2,9 @@
|
||||||
#define PLASMA_EXTENSION_H
|
#define PLASMA_EXTENSION_H
|
||||||
|
|
||||||
static int PyObjectToPlasmaConnection(PyObject *object,
|
static int PyObjectToPlasmaConnection(PyObject *object,
|
||||||
plasma_connection **conn) {
|
PlasmaConnection **conn) {
|
||||||
if (PyCapsule_IsValid(object, "plasma")) {
|
if (PyCapsule_IsValid(object, "plasma")) {
|
||||||
*conn = (plasma_connection *) PyCapsule_GetPointer(object, "plasma");
|
*conn = (PlasmaConnection *) PyCapsule_GetPointer(object, "plasma");
|
||||||
return 1;
|
return 1;
|
||||||
} else {
|
} else {
|
||||||
PyErr_SetString(PyExc_TypeError, "must be a 'plasma' capsule");
|
PyErr_SetString(PyExc_TypeError, "must be a 'plasma' capsule");
|
||||||
|
|
|
@ -43,8 +43,7 @@
|
||||||
* @param object_id ID of the object for which we process this request.
|
* @param object_id ID of the object for which we process this request.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void process_status_request(client_connection *client_conn,
|
void process_status_request(ClientConnection *client_conn, ObjectID object_id);
|
||||||
object_id object_id);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Request the transfer from a remote node or get the status of
|
* Request the transfer from a remote node or get the status of
|
||||||
|
@ -58,7 +57,7 @@ void process_status_request(client_connection *client_conn,
|
||||||
* @param context Client connection.
|
* @param context Client connection.
|
||||||
* @return Status of object_id as defined in plasma.h
|
* @return Status of object_id as defined in plasma.h
|
||||||
*/
|
*/
|
||||||
int request_status(object_id object_id,
|
int request_status(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context);
|
void *context);
|
||||||
|
@ -75,10 +74,10 @@ int request_status(object_id object_id,
|
||||||
* @param conn The client connection object.
|
* @param conn The client connection object.
|
||||||
*/
|
*/
|
||||||
void process_transfer_request(event_loop *loop,
|
void process_transfer_request(event_loop *loop,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
const char *addr,
|
const char *addr,
|
||||||
int port,
|
int port,
|
||||||
client_connection *conn);
|
ClientConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Receive object_id requested by this Plamsa Manager from the remote Plasma
|
* Receive object_id requested by this Plamsa Manager from the remote Plasma
|
||||||
|
@ -94,22 +93,22 @@ void process_transfer_request(event_loop *loop,
|
||||||
*/
|
*/
|
||||||
void process_data_request(event_loop *loop,
|
void process_data_request(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
client_connection *conn);
|
ClientConnection *conn);
|
||||||
|
|
||||||
/** Entry of the hashtable of objects that are available locally. */
|
/** Entry of the hashtable of objects that are available locally. */
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Object id of this object. */
|
/** Object id of this object. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} available_object;
|
} available_object;
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the object we are fetching or waiting for. */
|
/** The ID of the object we are fetching or waiting for. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** Pointer to the array containing the manager locations of this object. This
|
/** Pointer to the array containing the manager locations of this object. This
|
||||||
* struct owns and must free each entry. */
|
* struct owns and must free each entry. */
|
||||||
char **manager_vector;
|
char **manager_vector;
|
||||||
|
@ -146,7 +145,7 @@ typedef struct {
|
||||||
*/
|
*/
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The client connection that called wait. */
|
/** The client connection that called wait. */
|
||||||
client_connection *client_conn;
|
ClientConnection *client_conn;
|
||||||
/** The ID of the timer that will time out and cause this wait to return to
|
/** The ID of the timer that will time out and cause this wait to return to
|
||||||
* the client if it hasn't already returned. */
|
* the client if it hasn't already returned. */
|
||||||
int64_t timer;
|
int64_t timer;
|
||||||
|
@ -155,7 +154,7 @@ typedef struct {
|
||||||
/** The object requests for this wait request. Each object request has a
|
/** The object requests for this wait request. Each object request has a
|
||||||
* status field which is either PLASMA_QUERY_LOCAL or PLASMA_QUERY_ANYWHERE.
|
* status field which is either PLASMA_QUERY_LOCAL or PLASMA_QUERY_ANYWHERE.
|
||||||
*/
|
*/
|
||||||
object_request *object_requests;
|
ObjectRequest *object_requests;
|
||||||
/** The minimum number of objects to wait for in this request. */
|
/** The minimum number of objects to wait for in this request. */
|
||||||
int64_t num_objects_to_wait_for;
|
int64_t num_objects_to_wait_for;
|
||||||
/** The number of object requests in this wait request that are already
|
/** The number of object requests in this wait request that are already
|
||||||
|
@ -169,7 +168,7 @@ UT_icd wait_request_icd = {sizeof(wait_request *), NULL, NULL, NULL};
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the object. This is used as a key in a hash table. */
|
/** The ID of the object. This is used as a key in a hash table. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** An array of the wait requests involving this object ID. */
|
/** An array of the wait requests involving this object ID. */
|
||||||
UT_array *wait_requests;
|
UT_array *wait_requests;
|
||||||
/** Handle for the uthash table in the manager state that keeps track of the
|
/** Handle for the uthash table in the manager state that keeps track of the
|
||||||
|
@ -181,12 +180,12 @@ struct plasma_manager_state {
|
||||||
/** Event loop. */
|
/** Event loop. */
|
||||||
event_loop *loop;
|
event_loop *loop;
|
||||||
/** Connection to the local plasma store for reading or writing data. */
|
/** Connection to the local plasma store for reading or writing data. */
|
||||||
plasma_connection *plasma_conn;
|
PlasmaConnection *plasma_conn;
|
||||||
/** Hash table of all contexts for active connections to
|
/** Hash table of all contexts for active connections to
|
||||||
* other plasma managers. These are used for writing data to
|
* other plasma managers. These are used for writing data to
|
||||||
* other plasma stores. */
|
* other plasma stores. */
|
||||||
client_connection *manager_connections;
|
ClientConnection *manager_connections;
|
||||||
db_handle *db;
|
DBHandle *db;
|
||||||
/** Our address. */
|
/** Our address. */
|
||||||
const char *addr;
|
const char *addr;
|
||||||
/** Our port. */
|
/** Our port. */
|
||||||
|
@ -211,10 +210,10 @@ plasma_manager_state *g_manager_state = NULL;
|
||||||
/* The context for fetch and wait requests. These are per client, per object. */
|
/* The context for fetch and wait requests. These are per client, per object. */
|
||||||
struct client_object_request {
|
struct client_object_request {
|
||||||
/** The ID of the object we are fetching or waiting for. */
|
/** The ID of the object we are fetching or waiting for. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** The client connection context, shared between other
|
/** The client connection context, shared between other
|
||||||
* client_object_requests for the same client. */
|
* client_object_requests for the same client. */
|
||||||
client_connection *client_conn;
|
ClientConnection *client_conn;
|
||||||
/** The ID for the timer that will time out the current request to the state
|
/** The ID for the timer that will time out the current request to the state
|
||||||
* database or another plasma manager. */
|
* database or another plasma manager. */
|
||||||
int64_t timer;
|
int64_t timer;
|
||||||
|
@ -242,7 +241,7 @@ struct client_object_request {
|
||||||
};
|
};
|
||||||
|
|
||||||
/* Context for a client connection to another plasma manager. */
|
/* Context for a client connection to another plasma manager. */
|
||||||
struct client_connection {
|
struct ClientConnection {
|
||||||
/** Current state for this plasma manager. This is shared
|
/** Current state for this plasma manager. This is shared
|
||||||
* between all client connections to the plasma manager. */
|
* between all client connections to the plasma manager. */
|
||||||
plasma_manager_state *manager_state;
|
plasma_manager_state *manager_state;
|
||||||
|
@ -288,7 +287,7 @@ object_wait_requests **object_wait_requests_table_ptr_from_type(
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_wait_request_for_object(plasma_manager_state *manager_state,
|
void add_wait_request_for_object(plasma_manager_state *manager_state,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int type,
|
int type,
|
||||||
wait_request *wait_req) {
|
wait_request *wait_req) {
|
||||||
object_wait_requests **object_wait_requests_table_ptr =
|
object_wait_requests **object_wait_requests_table_ptr =
|
||||||
|
@ -312,7 +311,7 @@ void add_wait_request_for_object(plasma_manager_state *manager_state,
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_wait_request_for_object(plasma_manager_state *manager_state,
|
void remove_wait_request_for_object(plasma_manager_state *manager_state,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int type,
|
int type,
|
||||||
wait_request *wait_req) {
|
wait_request *wait_req) {
|
||||||
object_wait_requests **object_wait_requests_table_ptr =
|
object_wait_requests **object_wait_requests_table_ptr =
|
||||||
|
@ -368,7 +367,7 @@ void return_from_wait(plasma_manager_state *manager_state,
|
||||||
}
|
}
|
||||||
|
|
||||||
void update_object_wait_requests(plasma_manager_state *manager_state,
|
void update_object_wait_requests(plasma_manager_state *manager_state,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int type,
|
int type,
|
||||||
int status) {
|
int status) {
|
||||||
object_wait_requests **object_wait_requests_table_ptr =
|
object_wait_requests **object_wait_requests_table_ptr =
|
||||||
|
@ -394,7 +393,7 @@ void update_object_wait_requests(plasma_manager_state *manager_state,
|
||||||
/* Mark the object as present in the wait request. */
|
/* Mark the object as present in the wait request. */
|
||||||
int j = 0;
|
int j = 0;
|
||||||
for (; j < wait_req->num_object_requests; ++j) {
|
for (; j < wait_req->num_object_requests; ++j) {
|
||||||
if (object_ids_equal(wait_req->object_requests[j].object_id, obj_id)) {
|
if (ObjectID_equal(wait_req->object_requests[j].object_id, obj_id)) {
|
||||||
/* Check that this object is currently nonexistent. */
|
/* Check that this object is currently nonexistent. */
|
||||||
CHECK(wait_req->object_requests[j].status ==
|
CHECK(wait_req->object_requests[j].status ==
|
||||||
ObjectStatus_Nonexistent);
|
ObjectStatus_Nonexistent);
|
||||||
|
@ -423,7 +422,7 @@ void update_object_wait_requests(plasma_manager_state *manager_state,
|
||||||
}
|
}
|
||||||
|
|
||||||
fetch_request *create_fetch_request(plasma_manager_state *manager_state,
|
fetch_request *create_fetch_request(plasma_manager_state *manager_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
fetch_request *fetch_req = malloc(sizeof(fetch_request));
|
fetch_request *fetch_req = malloc(sizeof(fetch_request));
|
||||||
fetch_req->object_id = object_id;
|
fetch_req->object_id = object_id;
|
||||||
fetch_req->manager_count = 0;
|
fetch_req->manager_count = 0;
|
||||||
|
@ -496,7 +495,7 @@ plasma_manager_state *init_plasma_manager_state(const char *store_socket_name,
|
||||||
}
|
}
|
||||||
|
|
||||||
void destroy_plasma_manager_state(plasma_manager_state *state) {
|
void destroy_plasma_manager_state(plasma_manager_state *state) {
|
||||||
client_connection *manager_conn, *tmp;
|
ClientConnection *manager_conn, *tmp;
|
||||||
HASH_ITER(manager_hh, state->manager_connections, manager_conn, tmp) {
|
HASH_ITER(manager_hh, state->manager_connections, manager_conn, tmp) {
|
||||||
HASH_DELETE(manager_hh, state->manager_connections, manager_conn);
|
HASH_DELETE(manager_hh, state->manager_connections, manager_conn);
|
||||||
plasma_request_buffer *head = manager_conn->transfer_queue;
|
plasma_request_buffer *head = manager_conn->transfer_queue;
|
||||||
|
@ -534,7 +533,7 @@ void process_message(event_loop *loop,
|
||||||
void *context,
|
void *context,
|
||||||
int events);
|
int events);
|
||||||
|
|
||||||
void write_object_chunk(client_connection *conn, plasma_request_buffer *buf) {
|
void write_object_chunk(ClientConnection *conn, plasma_request_buffer *buf) {
|
||||||
LOG_DEBUG("Writing data to fd %d", conn->fd);
|
LOG_DEBUG("Writing data to fd %d", conn->fd);
|
||||||
ssize_t r, s;
|
ssize_t r, s;
|
||||||
/* Try to write one BUFSIZE at a time. */
|
/* Try to write one BUFSIZE at a time. */
|
||||||
|
@ -568,7 +567,7 @@ void send_queued_request(event_loop *loop,
|
||||||
int data_sock,
|
int data_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
client_connection *conn = (client_connection *) context;
|
ClientConnection *conn = (ClientConnection *) context;
|
||||||
plasma_manager_state *state = conn->manager_state;
|
plasma_manager_state *state = conn->manager_state;
|
||||||
|
|
||||||
if (conn->transfer_queue == NULL) {
|
if (conn->transfer_queue == NULL) {
|
||||||
|
@ -610,7 +609,7 @@ void send_queued_request(event_loop *loop,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
int read_object_chunk(client_connection *conn, plasma_request_buffer *buf) {
|
int read_object_chunk(ClientConnection *conn, plasma_request_buffer *buf) {
|
||||||
LOG_DEBUG("Reading data from fd %d to %p", conn->fd,
|
LOG_DEBUG("Reading data from fd %d to %p", conn->fd,
|
||||||
buf->data + conn->cursor);
|
buf->data + conn->cursor);
|
||||||
ssize_t r, s;
|
ssize_t r, s;
|
||||||
|
@ -644,7 +643,7 @@ void process_data_chunk(event_loop *loop,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
/* Read the object chunk. */
|
/* Read the object chunk. */
|
||||||
client_connection *conn = (client_connection *) context;
|
ClientConnection *conn = (ClientConnection *) context;
|
||||||
plasma_request_buffer *buf = conn->transfer_queue;
|
plasma_request_buffer *buf = conn->transfer_queue;
|
||||||
int done = read_object_chunk(conn, buf);
|
int done = read_object_chunk(conn, buf);
|
||||||
if (!done) {
|
if (!done) {
|
||||||
|
@ -672,7 +671,7 @@ void ignore_data_chunk(event_loop *loop,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
/* Read the object chunk. */
|
/* Read the object chunk. */
|
||||||
client_connection *conn = (client_connection *) context;
|
ClientConnection *conn = (ClientConnection *) context;
|
||||||
plasma_request_buffer *buf = conn->ignore_buffer;
|
plasma_request_buffer *buf = conn->ignore_buffer;
|
||||||
|
|
||||||
/* Just read the transferred data into ignore_buf and then drop (free) it. */
|
/* Just read the transferred data into ignore_buf and then drop (free) it. */
|
||||||
|
@ -689,15 +688,15 @@ void ignore_data_chunk(event_loop *loop,
|
||||||
event_loop_add_file(loop, data_sock, EVENT_LOOP_READ, process_message, conn);
|
event_loop_add_file(loop, data_sock, EVENT_LOOP_READ, process_message, conn);
|
||||||
}
|
}
|
||||||
|
|
||||||
client_connection *get_manager_connection(plasma_manager_state *state,
|
ClientConnection *get_manager_connection(plasma_manager_state *state,
|
||||||
const char *ip_addr,
|
const char *ip_addr,
|
||||||
int port) {
|
int port) {
|
||||||
/* TODO(swang): Should probably check whether ip_addr and port belong to us.
|
/* TODO(swang): Should probably check whether ip_addr and port belong to us.
|
||||||
*/
|
*/
|
||||||
UT_string *ip_addr_port;
|
UT_string *ip_addr_port;
|
||||||
utstring_new(ip_addr_port);
|
utstring_new(ip_addr_port);
|
||||||
utstring_printf(ip_addr_port, "%s:%d", ip_addr, port);
|
utstring_printf(ip_addr_port, "%s:%d", ip_addr, port);
|
||||||
client_connection *manager_conn;
|
ClientConnection *manager_conn;
|
||||||
HASH_FIND(manager_hh, state->manager_connections, utstring_body(ip_addr_port),
|
HASH_FIND(manager_hh, state->manager_connections, utstring_body(ip_addr_port),
|
||||||
utstring_len(ip_addr_port), manager_conn);
|
utstring_len(ip_addr_port), manager_conn);
|
||||||
if (!manager_conn) {
|
if (!manager_conn) {
|
||||||
|
@ -706,7 +705,7 @@ client_connection *get_manager_connection(plasma_manager_state *state,
|
||||||
/* TODO(swang): Handle the case when connection to this manager was
|
/* TODO(swang): Handle the case when connection to this manager was
|
||||||
* unsuccessful. */
|
* unsuccessful. */
|
||||||
CHECK(fd >= 0);
|
CHECK(fd >= 0);
|
||||||
manager_conn = malloc(sizeof(client_connection));
|
manager_conn = malloc(sizeof(ClientConnection));
|
||||||
manager_conn->fd = fd;
|
manager_conn->fd = fd;
|
||||||
manager_conn->manager_state = state;
|
manager_conn->manager_state = state;
|
||||||
manager_conn->transfer_queue = NULL;
|
manager_conn->transfer_queue = NULL;
|
||||||
|
@ -722,18 +721,18 @@ client_connection *get_manager_connection(plasma_manager_state *state,
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_transfer_request(event_loop *loop,
|
void process_transfer_request(event_loop *loop,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
const char *addr,
|
const char *addr,
|
||||||
int port,
|
int port,
|
||||||
client_connection *conn) {
|
ClientConnection *conn) {
|
||||||
client_connection *manager_conn =
|
ClientConnection *manager_conn =
|
||||||
get_manager_connection(conn->manager_state, addr, port);
|
get_manager_connection(conn->manager_state, addr, port);
|
||||||
|
|
||||||
/* If there is already a request in the transfer queue with the same object
|
/* If there is already a request in the transfer queue with the same object
|
||||||
* ID, do not add the transfer request. */
|
* ID, do not add the transfer request. */
|
||||||
plasma_request_buffer *pending;
|
plasma_request_buffer *pending;
|
||||||
LL_FOREACH(manager_conn->transfer_queue, pending) {
|
LL_FOREACH(manager_conn->transfer_queue, pending) {
|
||||||
if (object_ids_equal(pending->object_id, obj_id) &&
|
if (ObjectID_equal(pending->object_id, obj_id) &&
|
||||||
(pending->type == MessageType_PlasmaDataReply)) {
|
(pending->type == MessageType_PlasmaDataReply)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -760,11 +759,11 @@ void process_transfer_request(event_loop *loop,
|
||||||
* do a non-blocking get call on the store, and if the object isn't there then
|
* do a non-blocking get call on the store, and if the object isn't there then
|
||||||
* perhaps the manager should initiate the transfer when it receives a
|
* perhaps the manager should initiate the transfer when it receives a
|
||||||
* notification from the store that the object is present. */
|
* notification from the store that the object is present. */
|
||||||
object_buffer obj_buffer;
|
ObjectBuffer obj_buffer;
|
||||||
int counter = 0;
|
int counter = 0;
|
||||||
do {
|
do {
|
||||||
/* We pass in 0 to indicate that the command should return immediately. */
|
/* We pass in 0 to indicate that the command should return immediately. */
|
||||||
object_id obj_id_array[1] = {obj_id};
|
ObjectID obj_id_array[1] = {obj_id};
|
||||||
plasma_get(conn->manager_state->plasma_conn, obj_id_array, 1, 0,
|
plasma_get(conn->manager_state->plasma_conn, obj_id_array, 1, 0,
|
||||||
&obj_buffer);
|
&obj_buffer);
|
||||||
if (counter > 0) {
|
if (counter > 0) {
|
||||||
|
@ -800,10 +799,10 @@ void process_transfer_request(event_loop *loop,
|
||||||
*/
|
*/
|
||||||
void process_data_request(event_loop *loop,
|
void process_data_request(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
client_connection *conn) {
|
ClientConnection *conn) {
|
||||||
plasma_request_buffer *buf = malloc(sizeof(plasma_request_buffer));
|
plasma_request_buffer *buf = malloc(sizeof(plasma_request_buffer));
|
||||||
buf->object_id = object_id;
|
buf->object_id = object_id;
|
||||||
buf->data_size = data_size;
|
buf->data_size = data_size;
|
||||||
|
@ -842,7 +841,7 @@ void process_data_request(event_loop *loop,
|
||||||
}
|
}
|
||||||
|
|
||||||
void request_transfer_from(plasma_manager_state *manager_state,
|
void request_transfer_from(plasma_manager_state *manager_state,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
fetch_request *fetch_req;
|
fetch_request *fetch_req;
|
||||||
HASH_FIND(hh, manager_state->fetch_requests, &object_id, sizeof(object_id),
|
HASH_FIND(hh, manager_state->fetch_requests, &object_id, sizeof(object_id),
|
||||||
fetch_req);
|
fetch_req);
|
||||||
|
@ -858,7 +857,7 @@ void request_transfer_from(plasma_manager_state *manager_state,
|
||||||
parse_ip_addr_port(fetch_req->manager_vector[fetch_req->next_manager], addr,
|
parse_ip_addr_port(fetch_req->manager_vector[fetch_req->next_manager], addr,
|
||||||
&port);
|
&port);
|
||||||
|
|
||||||
client_connection *manager_conn =
|
ClientConnection *manager_conn =
|
||||||
get_manager_connection(manager_state, addr, port);
|
get_manager_connection(manager_state, addr, port);
|
||||||
|
|
||||||
/* Check that this manager isn't trying to request an object from itself.
|
/* Check that this manager isn't trying to request an object from itself.
|
||||||
|
@ -901,14 +900,14 @@ int fetch_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
return MANAGER_TIMEOUT;
|
return MANAGER_TIMEOUT;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool is_object_local(plasma_manager_state *state, object_id object_id) {
|
bool is_object_local(plasma_manager_state *state, ObjectID object_id) {
|
||||||
available_object *entry;
|
available_object *entry;
|
||||||
HASH_FIND(hh, state->local_available_objects, &object_id, sizeof(object_id),
|
HASH_FIND(hh, state->local_available_objects, &object_id, sizeof(object_id),
|
||||||
entry);
|
entry);
|
||||||
return entry != NULL;
|
return entry != NULL;
|
||||||
}
|
}
|
||||||
|
|
||||||
void request_transfer(object_id object_id,
|
void request_transfer(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -958,7 +957,7 @@ void request_transfer(object_id object_id,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* This method is only called from the tests. */
|
/* This method is only called from the tests. */
|
||||||
void call_request_transfer(object_id object_id,
|
void call_request_transfer(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -975,11 +974,11 @@ void call_request_transfer(object_id object_id,
|
||||||
request_transfer(object_id, manager_count, manager_vector, context);
|
request_transfer(object_id, manager_count, manager_vector, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void fatal_table_callback(object_id id, void *user_context, void *user_data) {
|
void fatal_table_callback(ObjectID id, void *user_context, void *user_data) {
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_present_callback(object_id object_id,
|
void object_present_callback(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
|
@ -995,7 +994,7 @@ void object_present_callback(object_id object_id,
|
||||||
|
|
||||||
/* This callback is used by both fetch and wait. Therefore, it may have to
|
/* This callback is used by both fetch and wait. Therefore, it may have to
|
||||||
* handle outstanding fetch and wait requests. */
|
* handle outstanding fetch and wait requests. */
|
||||||
void object_table_subscribe_callback(object_id object_id,
|
void object_table_subscribe_callback(ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
|
@ -1012,18 +1011,18 @@ void object_table_subscribe_callback(object_id object_id,
|
||||||
object_present_callback(object_id, manager_count, manager_vector, context);
|
object_present_callback(object_id, manager_count, manager_vector, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_fetch_requests(client_connection *client_conn,
|
void process_fetch_requests(ClientConnection *client_conn,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[]) {
|
ObjectID object_ids[]) {
|
||||||
plasma_manager_state *manager_state = client_conn->manager_state;
|
plasma_manager_state *manager_state = client_conn->manager_state;
|
||||||
|
|
||||||
int num_object_ids_to_request = 0;
|
int num_object_ids_to_request = 0;
|
||||||
/* This is allocating more space than necessary, but we do not know the exact
|
/* This is allocating more space than necessary, but we do not know the exact
|
||||||
* number of object IDs to request notifications for yet. */
|
* number of object IDs to request notifications for yet. */
|
||||||
object_id *object_ids_to_request = malloc(num_object_ids * sizeof(object_id));
|
ObjectID *object_ids_to_request = malloc(num_object_ids * sizeof(ObjectID));
|
||||||
|
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
for (int i = 0; i < num_object_ids; ++i) {
|
||||||
object_id obj_id = object_ids[i];
|
ObjectID obj_id = object_ids[i];
|
||||||
|
|
||||||
/* Check if this object is already present locally. If so, do nothing. */
|
/* Check if this object is already present locally. If so, do nothing. */
|
||||||
if (is_object_local(manager_state, obj_id)) {
|
if (is_object_local(manager_state, obj_id)) {
|
||||||
|
@ -1066,9 +1065,9 @@ int wait_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
return EVENT_LOOP_TIMER_DONE;
|
return EVENT_LOOP_TIMER_DONE;
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_wait_request(client_connection *client_conn,
|
void process_wait_request(ClientConnection *client_conn,
|
||||||
int num_object_requests,
|
int num_object_requests,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
uint64_t timeout_ms,
|
uint64_t timeout_ms,
|
||||||
int num_ready_objects) {
|
int num_ready_objects) {
|
||||||
CHECK(client_conn != NULL);
|
CHECK(client_conn != NULL);
|
||||||
|
@ -1081,7 +1080,7 @@ void process_wait_request(client_connection *client_conn,
|
||||||
wait_req->timer = -1;
|
wait_req->timer = -1;
|
||||||
wait_req->num_object_requests = num_object_requests;
|
wait_req->num_object_requests = num_object_requests;
|
||||||
wait_req->object_requests =
|
wait_req->object_requests =
|
||||||
malloc(num_object_requests * sizeof(object_request));
|
malloc(num_object_requests * sizeof(ObjectRequest));
|
||||||
for (int i = 0; i < num_object_requests; ++i) {
|
for (int i = 0; i < num_object_requests; ++i) {
|
||||||
wait_req->object_requests[i].object_id = object_requests[i].object_id;
|
wait_req->object_requests[i].object_id = object_requests[i].object_id;
|
||||||
wait_req->object_requests[i].type = object_requests[i].type;
|
wait_req->object_requests[i].type = object_requests[i].type;
|
||||||
|
@ -1093,11 +1092,11 @@ void process_wait_request(client_connection *client_conn,
|
||||||
int num_object_ids_to_request = 0;
|
int num_object_ids_to_request = 0;
|
||||||
/* This is allocating more space than necessary, but we do not know the exact
|
/* This is allocating more space than necessary, but we do not know the exact
|
||||||
* number of object IDs to request notifications for yet. */
|
* number of object IDs to request notifications for yet. */
|
||||||
object_id *object_ids_to_request =
|
ObjectID *object_ids_to_request =
|
||||||
malloc(num_object_requests * sizeof(object_id));
|
malloc(num_object_requests * sizeof(ObjectID));
|
||||||
|
|
||||||
for (int i = 0; i < num_object_requests; ++i) {
|
for (int i = 0; i < num_object_requests; ++i) {
|
||||||
object_id obj_id = object_requests[i].object_id;
|
ObjectID obj_id = object_requests[i].object_id;
|
||||||
|
|
||||||
/* Check if this object is already present locally. If so, mark the object
|
/* Check if this object is already present locally. If so, mark the object
|
||||||
* as present. */
|
* as present. */
|
||||||
|
@ -1162,11 +1161,11 @@ void process_wait_request(client_connection *client_conn,
|
||||||
* @param context Client connection.
|
* @param context Client connection.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void request_status_done(object_id object_id,
|
void request_status_done(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
client_connection *client_conn = (client_connection *) context;
|
ClientConnection *client_conn = (ClientConnection *) context;
|
||||||
int status =
|
int status =
|
||||||
request_status(object_id, manager_count, manager_vector, context);
|
request_status(object_id, manager_count, manager_vector, context);
|
||||||
warn_if_sigpipe(plasma_send_StatusReply(client_conn->fd,
|
warn_if_sigpipe(plasma_send_StatusReply(client_conn->fd,
|
||||||
|
@ -1175,11 +1174,11 @@ void request_status_done(object_id object_id,
|
||||||
client_conn->fd);
|
client_conn->fd);
|
||||||
}
|
}
|
||||||
|
|
||||||
int request_status(object_id object_id,
|
int request_status(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context) {
|
void *context) {
|
||||||
client_connection *client_conn = (client_connection *) context;
|
ClientConnection *client_conn = (ClientConnection *) context;
|
||||||
|
|
||||||
/* Return success immediately if we already have this object. */
|
/* Return success immediately if we already have this object. */
|
||||||
if (is_object_local(client_conn->manager_state, object_id)) {
|
if (is_object_local(client_conn->manager_state, object_id)) {
|
||||||
|
@ -1192,7 +1191,7 @@ int request_status(object_id object_id,
|
||||||
return (manager_count > 0 ? ObjectStatus_Remote : ObjectStatus_Nonexistent);
|
return (manager_count > 0 ? ObjectStatus_Remote : ObjectStatus_Nonexistent);
|
||||||
}
|
}
|
||||||
|
|
||||||
void object_table_lookup_fail_callback(object_id object_id,
|
void object_table_lookup_fail_callback(ObjectID object_id,
|
||||||
void *user_context,
|
void *user_context,
|
||||||
void *user_data) {
|
void *user_data) {
|
||||||
/* Fail for now. Later, we may want to send a ObjectStatus_Nonexistent to the
|
/* Fail for now. Later, we may want to send a ObjectStatus_Nonexistent to the
|
||||||
|
@ -1200,8 +1199,7 @@ void object_table_lookup_fail_callback(object_id object_id,
|
||||||
CHECK(0);
|
CHECK(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_status_request(client_connection *client_conn,
|
void process_status_request(ClientConnection *client_conn, ObjectID object_id) {
|
||||||
object_id object_id) {
|
|
||||||
/* Return success immediately if we already have this object. */
|
/* Return success immediately if we already have this object. */
|
||||||
if (is_object_local(client_conn->manager_state, object_id)) {
|
if (is_object_local(client_conn->manager_state, object_id)) {
|
||||||
int status = ObjectStatus_Local;
|
int status = ObjectStatus_Local;
|
||||||
|
@ -1227,8 +1225,8 @@ void process_status_request(client_connection *client_conn,
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_delete_object_notification(plasma_manager_state *state,
|
void process_delete_object_notification(plasma_manager_state *state,
|
||||||
object_info object_info) {
|
ObjectInfo object_info) {
|
||||||
object_id obj_id = object_info.obj_id;
|
ObjectID obj_id = object_info.obj_id;
|
||||||
available_object *entry;
|
available_object *entry;
|
||||||
HASH_FIND(hh, state->local_available_objects, &obj_id, sizeof(obj_id), entry);
|
HASH_FIND(hh, state->local_available_objects, &obj_id, sizeof(obj_id), entry);
|
||||||
if (entry != NULL) {
|
if (entry != NULL) {
|
||||||
|
@ -1248,12 +1246,12 @@ void process_delete_object_notification(plasma_manager_state *state,
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_add_object_notification(plasma_manager_state *state,
|
void process_add_object_notification(plasma_manager_state *state,
|
||||||
object_info object_info) {
|
ObjectInfo object_info) {
|
||||||
object_id obj_id = object_info.obj_id;
|
ObjectID obj_id = object_info.obj_id;
|
||||||
available_object *entry =
|
available_object *entry =
|
||||||
(available_object *) malloc(sizeof(available_object));
|
(available_object *) malloc(sizeof(available_object));
|
||||||
entry->object_id = obj_id;
|
entry->object_id = obj_id;
|
||||||
HASH_ADD(hh, state->local_available_objects, object_id, sizeof(object_id),
|
HASH_ADD(hh, state->local_available_objects, object_id, sizeof(ObjectID),
|
||||||
entry);
|
entry);
|
||||||
|
|
||||||
/* Add this object to the (redis) object table. */
|
/* Add this object to the (redis) object table. */
|
||||||
|
@ -1285,7 +1283,7 @@ void process_object_notification(event_loop *loop,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
plasma_manager_state *state = context;
|
plasma_manager_state *state = context;
|
||||||
object_info object_info;
|
ObjectInfo object_info;
|
||||||
/* Read the notification from Plasma. */
|
/* Read the notification from Plasma. */
|
||||||
int error =
|
int error =
|
||||||
read_bytes(client_sock, (uint8_t *) &object_info, sizeof(object_info));
|
read_bytes(client_sock, (uint8_t *) &object_info, sizeof(object_info));
|
||||||
|
@ -1310,7 +1308,7 @@ void process_message(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
client_connection *conn = (client_connection *) context;
|
ClientConnection *conn = (ClientConnection *) context;
|
||||||
|
|
||||||
int64_t length;
|
int64_t length;
|
||||||
int64_t type;
|
int64_t type;
|
||||||
|
@ -1320,7 +1318,7 @@ void process_message(event_loop *loop,
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MessageType_PlasmaDataRequest: {
|
case MessageType_PlasmaDataRequest: {
|
||||||
LOG_DEBUG("Processing data request");
|
LOG_DEBUG("Processing data request");
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
char *address;
|
char *address;
|
||||||
int port;
|
int port;
|
||||||
plasma_read_DataRequest(data, &object_id, &address, &port);
|
plasma_read_DataRequest(data, &object_id, &address, &port);
|
||||||
|
@ -1329,7 +1327,7 @@ void process_message(event_loop *loop,
|
||||||
} break;
|
} break;
|
||||||
case MessageType_PlasmaDataReply: {
|
case MessageType_PlasmaDataReply: {
|
||||||
LOG_DEBUG("Processing data reply");
|
LOG_DEBUG("Processing data reply");
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
int64_t object_size;
|
int64_t object_size;
|
||||||
int64_t metadata_size;
|
int64_t metadata_size;
|
||||||
plasma_read_DataReply(data, &object_id, &object_size, &metadata_size);
|
plasma_read_DataReply(data, &object_id, &object_size, &metadata_size);
|
||||||
|
@ -1339,7 +1337,7 @@ void process_message(event_loop *loop,
|
||||||
case MessageType_PlasmaFetchRequest: {
|
case MessageType_PlasmaFetchRequest: {
|
||||||
LOG_DEBUG("Processing fetch remote");
|
LOG_DEBUG("Processing fetch remote");
|
||||||
int64_t num_objects = plasma_read_FetchRequest_num_objects(data);
|
int64_t num_objects = plasma_read_FetchRequest_num_objects(data);
|
||||||
object_id *object_ids_to_fetch = malloc(num_objects * sizeof(object_id));
|
ObjectID *object_ids_to_fetch = malloc(num_objects * sizeof(ObjectID));
|
||||||
/* TODO(pcm): process_fetch_requests allocates an array of num_objects
|
/* TODO(pcm): process_fetch_requests allocates an array of num_objects
|
||||||
* object_ids too so these should be shared in the future. */
|
* object_ids too so these should be shared in the future. */
|
||||||
plasma_read_FetchRequest(data, object_ids_to_fetch, num_objects);
|
plasma_read_FetchRequest(data, object_ids_to_fetch, num_objects);
|
||||||
|
@ -1349,8 +1347,8 @@ void process_message(event_loop *loop,
|
||||||
case MessageType_PlasmaWaitRequest: {
|
case MessageType_PlasmaWaitRequest: {
|
||||||
LOG_DEBUG("Processing wait");
|
LOG_DEBUG("Processing wait");
|
||||||
int num_object_ids = plasma_read_WaitRequest_num_object_ids(data);
|
int num_object_ids = plasma_read_WaitRequest_num_object_ids(data);
|
||||||
object_request *object_requests =
|
ObjectRequest *object_requests =
|
||||||
malloc(num_object_ids * sizeof(object_request));
|
malloc(num_object_ids * sizeof(ObjectRequest));
|
||||||
int64_t timeout_ms;
|
int64_t timeout_ms;
|
||||||
int num_ready_objects;
|
int num_ready_objects;
|
||||||
plasma_read_WaitRequest(data, &object_requests[0], num_object_ids,
|
plasma_read_WaitRequest(data, &object_requests[0], num_object_ids,
|
||||||
|
@ -1363,7 +1361,7 @@ void process_message(event_loop *loop,
|
||||||
} break;
|
} break;
|
||||||
case MessageType_PlasmaStatusRequest: {
|
case MessageType_PlasmaStatusRequest: {
|
||||||
LOG_DEBUG("Processing status");
|
LOG_DEBUG("Processing status");
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
int64_t num_objects = plasma_read_StatusRequest_num_objects(data);
|
int64_t num_objects = plasma_read_StatusRequest_num_objects(data);
|
||||||
CHECK(num_objects == 1);
|
CHECK(num_objects == 1);
|
||||||
plasma_read_StatusRequest(data, &object_id, 1);
|
plasma_read_StatusRequest(data, &object_id, 1);
|
||||||
|
@ -1384,15 +1382,15 @@ void process_message(event_loop *loop,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* TODO(pcm): Split this into two methods: new_worker_connection
|
/* TODO(pcm): Split this into two methods: new_worker_connection
|
||||||
* and new_manager_connection and also split client_connection
|
* and new_manager_connection and also split ClientConnection
|
||||||
* into two structs, one for workers and one for other plasma managers. */
|
* into two structs, one for workers and one for other plasma managers. */
|
||||||
client_connection *new_client_connection(event_loop *loop,
|
ClientConnection *ClientConnection_init(event_loop *loop,
|
||||||
int listener_sock,
|
int listener_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
int new_socket = accept_client(listener_sock);
|
int new_socket = accept_client(listener_sock);
|
||||||
/* Create a new data connection context per client. */
|
/* Create a new data connection context per client. */
|
||||||
client_connection *conn = malloc(sizeof(client_connection));
|
ClientConnection *conn = malloc(sizeof(ClientConnection));
|
||||||
conn->manager_state = (plasma_manager_state *) context;
|
conn->manager_state = (plasma_manager_state *) context;
|
||||||
conn->cursor = 0;
|
conn->cursor = 0;
|
||||||
conn->transfer_queue = NULL;
|
conn->transfer_queue = NULL;
|
||||||
|
@ -1408,10 +1406,10 @@ void handle_new_client(event_loop *loop,
|
||||||
int listener_sock,
|
int listener_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
(void) new_client_connection(loop, listener_sock, context, events);
|
(void) ClientConnection_init(loop, listener_sock, context, events);
|
||||||
}
|
}
|
||||||
|
|
||||||
int get_client_sock(client_connection *conn) {
|
int get_client_sock(ClientConnection *conn) {
|
||||||
return conn->fd;
|
return conn->fd;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,7 @@
|
||||||
#define BUFSIZE 4096
|
#define BUFSIZE 4096
|
||||||
|
|
||||||
typedef struct plasma_manager_state plasma_manager_state;
|
typedef struct plasma_manager_state plasma_manager_state;
|
||||||
typedef struct client_connection client_connection;
|
typedef struct ClientConnection ClientConnection;
|
||||||
typedef struct client_object_request client_object_request;
|
typedef struct client_object_request client_object_request;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -63,7 +63,7 @@ void destroy_plasma_manager_state(plasma_manager_state *state);
|
||||||
* @param object_id The object_id of the object we will be sending.
|
* @param object_id The object_id of the object we will be sending.
|
||||||
* @param addr The IP address of the plasma manager to send the object to.
|
* @param addr The IP address of the plasma manager to send the object to.
|
||||||
* @param port The port of the plasma manager we are sending the object to.
|
* @param port The port of the plasma manager we are sending the object to.
|
||||||
* @param conn The client_connection to the other plasma manager.
|
* @param conn The ClientConnection to the other plasma manager.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*
|
*
|
||||||
* This establishes a connection to the remote manager if one doesn't already
|
* This establishes a connection to the remote manager if one doesn't already
|
||||||
|
@ -71,10 +71,10 @@ void destroy_plasma_manager_state(plasma_manager_state *state);
|
||||||
* manager.
|
* manager.
|
||||||
*/
|
*/
|
||||||
void process_transfer(event_loop *loop,
|
void process_transfer(event_loop *loop,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
uint8_t addr[4],
|
uint8_t addr[4],
|
||||||
int port,
|
int port,
|
||||||
client_connection *conn);
|
ClientConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Process a request from another object store manager to receive data.
|
* Process a request from another object store manager to receive data.
|
||||||
|
@ -84,7 +84,7 @@ void process_transfer(event_loop *loop,
|
||||||
* @param object_id The object_id of the object we will be reading.
|
* @param object_id The object_id of the object we will be reading.
|
||||||
* @param data_size Size of the object.
|
* @param data_size Size of the object.
|
||||||
* @param metadata_size Size of the metadata.
|
* @param metadata_size Size of the metadata.
|
||||||
* @param conn The client_connection to the other plasma manager.
|
* @param conn The ClientConnection to the other plasma manager.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*
|
*
|
||||||
* Initializes the object we are going to write to in the local plasma store
|
* Initializes the object we are going to write to in the local plasma store
|
||||||
|
@ -93,10 +93,10 @@ void process_transfer(event_loop *loop,
|
||||||
*/
|
*/
|
||||||
void process_data(event_loop *loop,
|
void process_data(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
client_connection *conn);
|
ClientConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read the next chunk of the object in transit from the plasma manager
|
* Read the next chunk of the object in transit from the plasma manager
|
||||||
|
@ -105,7 +105,7 @@ void process_data(event_loop *loop,
|
||||||
*
|
*
|
||||||
* @param loop This is the event loop of the plasma manager.
|
* @param loop This is the event loop of the plasma manager.
|
||||||
* @param data_sock The connection to the other plasma manager.
|
* @param data_sock The connection to the other plasma manager.
|
||||||
* @param context The client_connection to the other plasma manager.
|
* @param context The ClientConnection to the other plasma manager.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void process_data_chunk(event_loop *loop,
|
void process_data_chunk(event_loop *loop,
|
||||||
|
@ -136,7 +136,7 @@ void process_object_notification(event_loop *loop,
|
||||||
*
|
*
|
||||||
* @param loop This is the event loop of the plasma manager.
|
* @param loop This is the event loop of the plasma manager.
|
||||||
* @param data_sock This is the socket the other plasma manager is listening on.
|
* @param data_sock This is the socket the other plasma manager is listening on.
|
||||||
* @param context The client_connection to the other plasma manager, contains a
|
* @param context The ClientConnection to the other plasma manager, contains a
|
||||||
* queue of objects that will be sent.
|
* queue of objects that will be sent.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
|
@ -154,10 +154,10 @@ void send_queued_request(event_loop *loop,
|
||||||
* @param context The plasma manager state.
|
* @param context The plasma manager state.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
client_connection *new_client_connection(event_loop *loop,
|
ClientConnection *ClientConnection_init(event_loop *loop,
|
||||||
int listener_sock,
|
int listener_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events);
|
int events);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The following definitions are internal to the plasma manager code but are
|
* The following definitions are internal to the plasma manager code but are
|
||||||
|
@ -170,7 +170,7 @@ client_connection *new_client_connection(event_loop *loop,
|
||||||
typedef struct plasma_request_buffer plasma_request_buffer;
|
typedef struct plasma_request_buffer plasma_request_buffer;
|
||||||
struct plasma_request_buffer {
|
struct plasma_request_buffer {
|
||||||
int type;
|
int type;
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
uint8_t *data;
|
uint8_t *data;
|
||||||
int64_t data_size;
|
int64_t data_size;
|
||||||
uint8_t *metadata;
|
uint8_t *metadata;
|
||||||
|
@ -193,7 +193,7 @@ struct plasma_request_buffer {
|
||||||
* @param context The plasma manager state.
|
* @param context The plasma manager state.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void call_request_transfer(object_id object_id,
|
void call_request_transfer(ObjectID object_id,
|
||||||
int manager_count,
|
int manager_count,
|
||||||
const char *manager_vector[],
|
const char *manager_vector[],
|
||||||
void *context);
|
void *context);
|
||||||
|
@ -214,7 +214,7 @@ int fetch_timeout_handler(event_loop *loop, timer_id id, void *context);
|
||||||
* @param object_id The object ID whose context we want to delete.
|
* @param object_id The object ID whose context we want to delete.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void remove_object_request(client_connection *client_conn,
|
void remove_object_request(ClientConnection *client_conn,
|
||||||
client_object_request *object_req);
|
client_object_request *object_req);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -226,9 +226,9 @@ void remove_object_request(client_connection *client_conn,
|
||||||
* @param port The port that the remote manager is listening on.
|
* @param port The port that the remote manager is listening on.
|
||||||
* @return A pointer to the connection to the remote manager.
|
* @return A pointer to the connection to the remote manager.
|
||||||
*/
|
*/
|
||||||
client_connection *get_manager_connection(plasma_manager_state *state,
|
ClientConnection *get_manager_connection(plasma_manager_state *state,
|
||||||
const char *ip_addr,
|
const char *ip_addr,
|
||||||
int port);
|
int port);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads an object chunk sent by the given client into a buffer. This is the
|
* Reads an object chunk sent by the given client into a buffer. This is the
|
||||||
|
@ -240,7 +240,7 @@ client_connection *get_manager_connection(plasma_manager_state *state,
|
||||||
* object. 1 means that the client has sent all the data, 0 means there
|
* object. 1 means that the client has sent all the data, 0 means there
|
||||||
* is more.
|
* is more.
|
||||||
*/
|
*/
|
||||||
int read_object_chunk(client_connection *conn, plasma_request_buffer *buf);
|
int read_object_chunk(ClientConnection *conn, plasma_request_buffer *buf);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Writes an object chunk from a buffer to the given client. This is the
|
* Writes an object chunk from a buffer to the given client. This is the
|
||||||
|
@ -250,7 +250,7 @@ int read_object_chunk(client_connection *conn, plasma_request_buffer *buf);
|
||||||
* @param buf The buffer to read data from.
|
* @param buf The buffer to read data from.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void write_object_chunk(client_connection *conn, plasma_request_buffer *buf);
|
void write_object_chunk(ClientConnection *conn, plasma_request_buffer *buf);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the event loop of the given plasma manager state.
|
* Get the event loop of the given plasma manager state.
|
||||||
|
@ -267,7 +267,7 @@ event_loop *get_event_loop(plasma_manager_state *state);
|
||||||
* @param conn The connection to the client who's sending or reading data.
|
* @param conn The connection to the client who's sending or reading data.
|
||||||
* @return A file descriptor for the socket.
|
* @return A file descriptor for the socket.
|
||||||
*/
|
*/
|
||||||
int get_client_sock(client_connection *conn);
|
int get_client_sock(ClientConnection *conn);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return whether or not the object is local.
|
* Return whether or not the object is local.
|
||||||
|
@ -277,6 +277,6 @@ int get_client_sock(client_connection *conn);
|
||||||
* @return A bool that is true if the requested object is local and false
|
* @return A bool that is true if the requested object is local and false
|
||||||
* otherwise.
|
* otherwise.
|
||||||
*/
|
*/
|
||||||
bool is_object_local(plasma_manager_state *state, object_id object_id);
|
bool is_object_local(plasma_manager_state *state, ObjectID object_id);
|
||||||
|
|
||||||
#endif /* PLASMA_MANAGER_H */
|
#endif /* PLASMA_MANAGER_H */
|
||||||
|
|
|
@ -30,7 +30,7 @@ void free_protocol_builder(protocol_builder *builder) {
|
||||||
* @return Reference to the flatbuffer string vector.
|
* @return Reference to the flatbuffer string vector.
|
||||||
*/
|
*/
|
||||||
flatbuffers_string_vec_ref_t object_ids_to_flatbuffer(flatcc_builder_t *B,
|
flatbuffers_string_vec_ref_t object_ids_to_flatbuffer(flatcc_builder_t *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
flatbuffers_string_vec_start(B);
|
flatbuffers_string_vec_start(B);
|
||||||
for (int i = 0; i < num_objects; i++) {
|
for (int i = 0; i < num_objects; i++) {
|
||||||
|
@ -52,7 +52,7 @@ flatbuffers_string_vec_ref_t object_ids_to_flatbuffer(flatcc_builder_t *B,
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void object_ids_from_flatbuffer(flatbuffers_string_vec_t object_id_vector,
|
void object_ids_from_flatbuffer(flatbuffers_string_vec_t object_id_vector,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
CHECK(flatbuffers_string_vec_len(object_id_vector) == num_objects);
|
CHECK(flatbuffers_string_vec_len(object_id_vector) == num_objects);
|
||||||
for (int64_t i = 0; i < num_objects; ++i) {
|
for (int64_t i = 0; i < num_objects; ++i) {
|
||||||
|
@ -92,7 +92,7 @@ uint8_t *plasma_receive(int sock, int64_t message_type) {
|
||||||
|
|
||||||
int plasma_send_CreateRequest(int sock,
|
int plasma_send_CreateRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size) {
|
int64_t metadata_size) {
|
||||||
PlasmaCreateRequest_start_as_root(B);
|
PlasmaCreateRequest_start_as_root(B);
|
||||||
|
@ -105,7 +105,7 @@ int plasma_send_CreateRequest(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_CreateRequest(uint8_t *data,
|
void plasma_read_CreateRequest(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int64_t *data_size,
|
int64_t *data_size,
|
||||||
int64_t *metadata_size) {
|
int64_t *metadata_size) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
|
@ -119,8 +119,8 @@ void plasma_read_CreateRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_CreateReply(int sock,
|
int plasma_send_CreateReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
plasma_object *object,
|
PlasmaObject *object,
|
||||||
int error_code) {
|
int error_code) {
|
||||||
PlasmaCreateReply_start_as_root(B);
|
PlasmaCreateReply_start_as_root(B);
|
||||||
PlasmaCreateReply_object_id_create(B, (const char *) &object_id.id[0],
|
PlasmaCreateReply_object_id_create(B, (const char *) &object_id.id[0],
|
||||||
|
@ -134,27 +134,27 @@ int plasma_send_CreateReply(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_CreateReply(uint8_t *data,
|
void plasma_read_CreateReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
plasma_object *object,
|
PlasmaObject *object,
|
||||||
int *error_code) {
|
int *error_code) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaCreateReply_table_t rep = PlasmaCreateReply_as_root(data);
|
PlasmaCreateReply_table_t rep = PlasmaCreateReply_as_root(data);
|
||||||
flatbuffers_string_t id = PlasmaCreateReply_object_id(rep);
|
flatbuffers_string_t id = PlasmaCreateReply_object_id(rep);
|
||||||
CHECK(flatbuffers_string_len(id) == sizeof(object_id->id));
|
CHECK(flatbuffers_string_len(id) == sizeof(object_id->id));
|
||||||
memcpy(&object_id->id[0], id, sizeof(object_id->id));
|
memcpy(&object_id->id[0], id, sizeof(object_id->id));
|
||||||
PlasmaObject_struct_t obj = PlasmaCreateReply_plasma_object(rep);
|
PlasmaObjectSpec_struct_t obj = PlasmaCreateReply_plasma_object(rep);
|
||||||
object->handle.store_fd = PlasmaObject_segment_index(obj);
|
object->handle.store_fd = PlasmaObjectSpec_segment_index(obj);
|
||||||
object->handle.mmap_size = PlasmaObject_mmap_size(obj);
|
object->handle.mmap_size = PlasmaObjectSpec_mmap_size(obj);
|
||||||
object->data_offset = PlasmaObject_data_offset(obj);
|
object->data_offset = PlasmaObjectSpec_data_offset(obj);
|
||||||
object->data_size = PlasmaObject_data_size(obj);
|
object->data_size = PlasmaObjectSpec_data_size(obj);
|
||||||
object->metadata_offset = PlasmaObject_metadata_offset(obj);
|
object->metadata_offset = PlasmaObjectSpec_metadata_offset(obj);
|
||||||
object->metadata_size = PlasmaObject_metadata_size(obj);
|
object->metadata_size = PlasmaObjectSpec_metadata_size(obj);
|
||||||
*error_code = PlasmaCreateReply_error(rep);
|
*error_code = PlasmaCreateReply_error(rep);
|
||||||
}
|
}
|
||||||
|
|
||||||
#define DEFINE_SIMPLE_SEND_REQUEST(MESSAGE_NAME) \
|
#define DEFINE_SIMPLE_SEND_REQUEST(MESSAGE_NAME) \
|
||||||
int plasma_send_##MESSAGE_NAME(int sock, protocol_builder *B, \
|
int plasma_send_##MESSAGE_NAME(int sock, protocol_builder *B, \
|
||||||
object_id object_id) { \
|
ObjectID object_id) { \
|
||||||
Plasma##MESSAGE_NAME##_start_as_root(B); \
|
Plasma##MESSAGE_NAME##_start_as_root(B); \
|
||||||
Plasma##MESSAGE_NAME##_object_id_create( \
|
Plasma##MESSAGE_NAME##_object_id_create( \
|
||||||
B, (const char *) &object_id.id[0], sizeof(object_id.id)); \
|
B, (const char *) &object_id.id[0], sizeof(object_id.id)); \
|
||||||
|
@ -164,7 +164,7 @@ void plasma_read_CreateReply(uint8_t *data,
|
||||||
}
|
}
|
||||||
|
|
||||||
#define DEFINE_SIMPLE_READ_REQUEST(MESSAGE_NAME) \
|
#define DEFINE_SIMPLE_READ_REQUEST(MESSAGE_NAME) \
|
||||||
void plasma_read_##MESSAGE_NAME(uint8_t *data, object_id *object_id) { \
|
void plasma_read_##MESSAGE_NAME(uint8_t *data, ObjectID *object_id) { \
|
||||||
DCHECK(data); \
|
DCHECK(data); \
|
||||||
Plasma##MESSAGE_NAME##_table_t req = Plasma##MESSAGE_NAME##_as_root(data); \
|
Plasma##MESSAGE_NAME##_table_t req = Plasma##MESSAGE_NAME##_as_root(data); \
|
||||||
flatbuffers_string_t id = Plasma##MESSAGE_NAME##_object_id(req); \
|
flatbuffers_string_t id = Plasma##MESSAGE_NAME##_object_id(req); \
|
||||||
|
@ -174,7 +174,7 @@ void plasma_read_CreateReply(uint8_t *data,
|
||||||
|
|
||||||
#define DEFINE_SIMPLE_SEND_REPLY(MESSAGE_NAME) \
|
#define DEFINE_SIMPLE_SEND_REPLY(MESSAGE_NAME) \
|
||||||
int plasma_send_##MESSAGE_NAME(int sock, protocol_builder *B, \
|
int plasma_send_##MESSAGE_NAME(int sock, protocol_builder *B, \
|
||||||
object_id object_id, int error) { \
|
ObjectID object_id, int error) { \
|
||||||
Plasma##MESSAGE_NAME##_start_as_root(B); \
|
Plasma##MESSAGE_NAME##_start_as_root(B); \
|
||||||
Plasma##MESSAGE_NAME##_object_id_create( \
|
Plasma##MESSAGE_NAME##_object_id_create( \
|
||||||
B, (const char *) &object_id.id[0], sizeof(object_id.id)); \
|
B, (const char *) &object_id.id[0], sizeof(object_id.id)); \
|
||||||
|
@ -185,7 +185,7 @@ void plasma_read_CreateReply(uint8_t *data,
|
||||||
}
|
}
|
||||||
|
|
||||||
#define DEFINE_SIMPLE_READ_REPLY(MESSAGE_NAME) \
|
#define DEFINE_SIMPLE_READ_REPLY(MESSAGE_NAME) \
|
||||||
void plasma_read_##MESSAGE_NAME(uint8_t *data, object_id *object_id, \
|
void plasma_read_##MESSAGE_NAME(uint8_t *data, ObjectID *object_id, \
|
||||||
int *error) { \
|
int *error) { \
|
||||||
DCHECK(data); \
|
DCHECK(data); \
|
||||||
Plasma##MESSAGE_NAME##_table_t req = Plasma##MESSAGE_NAME##_as_root(data); \
|
Plasma##MESSAGE_NAME##_table_t req = Plasma##MESSAGE_NAME##_as_root(data); \
|
||||||
|
@ -197,7 +197,7 @@ void plasma_read_CreateReply(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_SealRequest(int sock,
|
int plasma_send_SealRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
unsigned char *digest) {
|
unsigned char *digest) {
|
||||||
PlasmaSealRequest_start_as_root(B);
|
PlasmaSealRequest_start_as_root(B);
|
||||||
PlasmaSealRequest_object_id_create(B, (const char *) &object_id.id[0],
|
PlasmaSealRequest_object_id_create(B, (const char *) &object_id.id[0],
|
||||||
|
@ -208,7 +208,7 @@ int plasma_send_SealRequest(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_SealRequest(uint8_t *data,
|
void plasma_read_SealRequest(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
unsigned char *digest) {
|
unsigned char *digest) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaSealRequest_table_t req = PlasmaSealRequest_as_root(data);
|
PlasmaSealRequest_table_t req = PlasmaSealRequest_as_root(data);
|
||||||
|
@ -237,7 +237,7 @@ DEFINE_SIMPLE_READ_REPLY(DeleteReply);
|
||||||
|
|
||||||
int plasma_send_StatusRequest(int sock,
|
int plasma_send_StatusRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
PlasmaStatusRequest_start_as_root(B);
|
PlasmaStatusRequest_start_as_root(B);
|
||||||
PlasmaStatusRequest_object_ids_add(
|
PlasmaStatusRequest_object_ids_add(
|
||||||
|
@ -253,7 +253,7 @@ int64_t plasma_read_StatusRequest_num_objects(uint8_t *data) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_StatusRequest(uint8_t *data,
|
void plasma_read_StatusRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaStatusRequest_table_t req = PlasmaStatusRequest_as_root(data);
|
PlasmaStatusRequest_table_t req = PlasmaStatusRequest_as_root(data);
|
||||||
|
@ -263,7 +263,7 @@ void plasma_read_StatusRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_StatusReply(int sock,
|
int plasma_send_StatusReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int object_status[],
|
int object_status[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
PlasmaStatusReply_start_as_root(B);
|
PlasmaStatusReply_start_as_root(B);
|
||||||
|
@ -285,7 +285,7 @@ int64_t plasma_read_StatusReply_num_objects(uint8_t *data) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_StatusReply(uint8_t *data,
|
void plasma_read_StatusReply(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int object_status[],
|
int object_status[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
|
@ -302,7 +302,7 @@ void plasma_read_StatusReply(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_ContainsRequest(int sock,
|
int plasma_send_ContainsRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id) {
|
ObjectID object_id) {
|
||||||
PlasmaContainsRequest_start_as_root(B);
|
PlasmaContainsRequest_start_as_root(B);
|
||||||
PlasmaContainsRequest_object_id_create(B, (const char *) &object_id.id[0],
|
PlasmaContainsRequest_object_id_create(B, (const char *) &object_id.id[0],
|
||||||
sizeof(object_id.id));
|
sizeof(object_id.id));
|
||||||
|
@ -310,7 +310,7 @@ int plasma_send_ContainsRequest(int sock,
|
||||||
return finalize_buffer_and_send(B, sock, MessageType_PlasmaContainsRequest);
|
return finalize_buffer_and_send(B, sock, MessageType_PlasmaContainsRequest);
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_ContainsRequest(uint8_t *data, object_id *object_id) {
|
void plasma_read_ContainsRequest(uint8_t *data, ObjectID *object_id) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaContainsRequest_table_t req = PlasmaContainsRequest_as_root(data);
|
PlasmaContainsRequest_table_t req = PlasmaContainsRequest_as_root(data);
|
||||||
flatbuffers_string_t id = PlasmaContainsRequest_object_id(req);
|
flatbuffers_string_t id = PlasmaContainsRequest_object_id(req);
|
||||||
|
@ -320,7 +320,7 @@ void plasma_read_ContainsRequest(uint8_t *data, object_id *object_id) {
|
||||||
|
|
||||||
int plasma_send_ContainsReply(int sock,
|
int plasma_send_ContainsReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int has_object) {
|
int has_object) {
|
||||||
PlasmaContainsReply_start_as_root(B);
|
PlasmaContainsReply_start_as_root(B);
|
||||||
PlasmaContainsReply_object_id_create(B, (const char *) &object_id.id[0],
|
PlasmaContainsReply_object_id_create(B, (const char *) &object_id.id[0],
|
||||||
|
@ -331,7 +331,7 @@ int plasma_send_ContainsReply(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_ContainsReply(uint8_t *data,
|
void plasma_read_ContainsReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int *has_object) {
|
int *has_object) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaContainsReply_table_t rep = PlasmaContainsReply_as_root(data);
|
PlasmaContainsReply_table_t rep = PlasmaContainsReply_as_root(data);
|
||||||
|
@ -401,7 +401,7 @@ void plasma_read_EvictReply(uint8_t *data, int64_t *num_bytes) {
|
||||||
|
|
||||||
int plasma_send_GetRequest(int sock,
|
int plasma_send_GetRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects,
|
int64_t num_objects,
|
||||||
int64_t timeout_ms) {
|
int64_t timeout_ms) {
|
||||||
PlasmaGetRequest_start_as_root(B);
|
PlasmaGetRequest_start_as_root(B);
|
||||||
|
@ -419,7 +419,7 @@ int64_t plasma_read_GetRequest_num_objects(uint8_t *data) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_GetRequest(uint8_t *data,
|
void plasma_read_GetRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t *timeout_ms,
|
int64_t *timeout_ms,
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
|
@ -431,8 +431,8 @@ void plasma_read_GetRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_GetReply(int sock,
|
int plasma_send_GetReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
plasma_object plasma_objects[],
|
PlasmaObject plasma_objects[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
PlasmaGetReply_start_as_root(B);
|
PlasmaGetReply_start_as_root(B);
|
||||||
|
|
||||||
|
@ -440,45 +440,47 @@ int plasma_send_GetReply(int sock,
|
||||||
object_ids_to_flatbuffer(B, object_ids, num_objects);
|
object_ids_to_flatbuffer(B, object_ids, num_objects);
|
||||||
PlasmaGetReply_object_ids_add(B, ids);
|
PlasmaGetReply_object_ids_add(B, ids);
|
||||||
|
|
||||||
PlasmaObject_vec_start(B);
|
PlasmaObjectSpec_vec_start(B);
|
||||||
for (int i = 0; i < num_objects; ++i) {
|
for (int i = 0; i < num_objects; ++i) {
|
||||||
plasma_object obj = plasma_objects[i];
|
PlasmaObject obj = plasma_objects[i];
|
||||||
PlasmaObject_t plasma_obj;
|
PlasmaObjectSpec_t plasma_obj;
|
||||||
memset(&plasma_obj, 0, sizeof(PlasmaObject_t));
|
memset(&plasma_obj, 0, sizeof(PlasmaObjectSpec_t));
|
||||||
plasma_obj.segment_index = obj.handle.store_fd;
|
plasma_obj.segment_index = obj.handle.store_fd;
|
||||||
plasma_obj.mmap_size = obj.handle.mmap_size;
|
plasma_obj.mmap_size = obj.handle.mmap_size;
|
||||||
plasma_obj.data_offset = obj.data_offset;
|
plasma_obj.data_offset = obj.data_offset;
|
||||||
plasma_obj.data_size = obj.data_size;
|
plasma_obj.data_size = obj.data_size;
|
||||||
plasma_obj.metadata_offset = obj.metadata_offset;
|
plasma_obj.metadata_offset = obj.metadata_offset;
|
||||||
plasma_obj.metadata_size = obj.metadata_size;
|
plasma_obj.metadata_size = obj.metadata_size;
|
||||||
PlasmaObject_vec_push(B, &plasma_obj);
|
PlasmaObjectSpec_vec_push(B, &plasma_obj);
|
||||||
}
|
}
|
||||||
PlasmaObject_vec_ref_t object_vec = PlasmaObject_vec_end(B);
|
PlasmaObjectSpec_vec_ref_t object_vec = PlasmaObjectSpec_vec_end(B);
|
||||||
PlasmaGetReply_plasma_objects_add(B, object_vec);
|
PlasmaGetReply_plasma_objects_add(B, object_vec);
|
||||||
PlasmaGetReply_end_as_root(B);
|
PlasmaGetReply_end_as_root(B);
|
||||||
return finalize_buffer_and_send(B, sock, MessageType_PlasmaGetReply);
|
return finalize_buffer_and_send(B, sock, MessageType_PlasmaGetReply);
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_GetReply(uint8_t *data,
|
void plasma_read_GetReply(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
plasma_object plasma_objects[],
|
PlasmaObject plasma_objects[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
CHECK(data);
|
CHECK(data);
|
||||||
PlasmaGetReply_table_t req = PlasmaGetReply_as_root(data);
|
PlasmaGetReply_table_t req = PlasmaGetReply_as_root(data);
|
||||||
flatbuffers_string_vec_t object_id_vector = PlasmaGetReply_object_ids(req);
|
flatbuffers_string_vec_t object_id_vector = PlasmaGetReply_object_ids(req);
|
||||||
object_ids_from_flatbuffer(object_id_vector, object_ids, num_objects);
|
object_ids_from_flatbuffer(object_id_vector, object_ids, num_objects);
|
||||||
|
|
||||||
memset(plasma_objects, 0, sizeof(plasma_object) * num_objects);
|
memset(plasma_objects, 0, sizeof(PlasmaObject) * num_objects);
|
||||||
PlasmaObject_vec_t plasma_objects_vector = PlasmaGetReply_plasma_objects(req);
|
PlasmaObjectSpec_vec_t plasma_objects_vector =
|
||||||
|
PlasmaGetReply_plasma_objects(req);
|
||||||
|
|
||||||
for (int i = 0; i < num_objects; ++i) {
|
for (int i = 0; i < num_objects; ++i) {
|
||||||
PlasmaObject_struct_t obj = PlasmaObject_vec_at(plasma_objects_vector, i);
|
PlasmaObjectSpec_struct_t obj =
|
||||||
plasma_objects[i].handle.store_fd = PlasmaObject_segment_index(obj);
|
PlasmaObjectSpec_vec_at(plasma_objects_vector, i);
|
||||||
plasma_objects[i].handle.mmap_size = PlasmaObject_mmap_size(obj);
|
plasma_objects[i].handle.store_fd = PlasmaObjectSpec_segment_index(obj);
|
||||||
plasma_objects[i].data_offset = PlasmaObject_data_offset(obj);
|
plasma_objects[i].handle.mmap_size = PlasmaObjectSpec_mmap_size(obj);
|
||||||
plasma_objects[i].data_size = PlasmaObject_data_size(obj);
|
plasma_objects[i].data_offset = PlasmaObjectSpec_data_offset(obj);
|
||||||
plasma_objects[i].metadata_offset = PlasmaObject_metadata_offset(obj);
|
plasma_objects[i].data_size = PlasmaObjectSpec_data_size(obj);
|
||||||
plasma_objects[i].metadata_size = PlasmaObject_metadata_size(obj);
|
plasma_objects[i].metadata_offset = PlasmaObjectSpec_metadata_offset(obj);
|
||||||
|
plasma_objects[i].metadata_size = PlasmaObjectSpec_metadata_size(obj);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -486,7 +488,7 @@ void plasma_read_GetReply(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_FetchRequest(int sock,
|
int plasma_send_FetchRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
PlasmaFetchRequest_start_as_root(B);
|
PlasmaFetchRequest_start_as_root(B);
|
||||||
PlasmaFetchRequest_object_ids_add(
|
PlasmaFetchRequest_object_ids_add(
|
||||||
|
@ -502,7 +504,7 @@ int64_t plasma_read_FetchRequest_num_objects(uint8_t *data) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_FetchRequest(uint8_t *data,
|
void plasma_read_FetchRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects) {
|
int64_t num_objects) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaFetchRequest_table_t req = PlasmaFetchRequest_as_root(data);
|
PlasmaFetchRequest_table_t req = PlasmaFetchRequest_as_root(data);
|
||||||
|
@ -515,7 +517,7 @@ void plasma_read_FetchRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_WaitRequest(int sock,
|
int plasma_send_WaitRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_requests,
|
int num_requests,
|
||||||
int num_ready_objects,
|
int num_ready_objects,
|
||||||
int64_t timeout_ms) {
|
int64_t timeout_ms) {
|
||||||
|
@ -542,7 +544,7 @@ int plasma_read_WaitRequest_num_object_ids(uint8_t *data) {
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_WaitRequest(uint8_t *data,
|
void plasma_read_WaitRequest(uint8_t *data,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
int64_t *timeout_ms,
|
int64_t *timeout_ms,
|
||||||
int *num_ready_objects) {
|
int *num_ready_objects) {
|
||||||
|
@ -562,7 +564,7 @@ void plasma_read_WaitRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_WaitReply(int sock,
|
int plasma_send_WaitReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_ready_objects) {
|
int num_ready_objects) {
|
||||||
PlasmaWaitReply_start_as_root(B);
|
PlasmaWaitReply_start_as_root(B);
|
||||||
ObjectReply_vec_start(B);
|
ObjectReply_vec_start(B);
|
||||||
|
@ -580,7 +582,7 @@ int plasma_send_WaitReply(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_WaitReply(uint8_t *data,
|
void plasma_read_WaitReply(uint8_t *data,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int *num_ready_objects) {
|
int *num_ready_objects) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
PlasmaWaitReply_table_t req = PlasmaWaitReply_as_root(data);
|
PlasmaWaitReply_table_t req = PlasmaWaitReply_as_root(data);
|
||||||
|
@ -604,7 +606,7 @@ int plasma_send_SubscribeRequest(int sock, protocol_builder *B) {
|
||||||
|
|
||||||
int plasma_send_DataRequest(int sock,
|
int plasma_send_DataRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
const char *address,
|
const char *address,
|
||||||
int port) {
|
int port) {
|
||||||
PlasmaDataRequest_start_as_root(B);
|
PlasmaDataRequest_start_as_root(B);
|
||||||
|
@ -617,7 +619,7 @@ int plasma_send_DataRequest(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_DataRequest(uint8_t *data,
|
void plasma_read_DataRequest(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
char **address,
|
char **address,
|
||||||
int *port) {
|
int *port) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
|
@ -631,7 +633,7 @@ void plasma_read_DataRequest(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_DataReply(int sock,
|
int plasma_send_DataReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t object_size,
|
int64_t object_size,
|
||||||
int64_t metadata_size) {
|
int64_t metadata_size) {
|
||||||
PlasmaDataReply_start_as_root(B);
|
PlasmaDataReply_start_as_root(B);
|
||||||
|
@ -644,7 +646,7 @@ int plasma_send_DataReply(int sock,
|
||||||
}
|
}
|
||||||
|
|
||||||
void plasma_read_DataReply(uint8_t *data,
|
void plasma_read_DataReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int64_t *object_size,
|
int64_t *object_size,
|
||||||
int64_t *metadata_size) {
|
int64_t *metadata_size) {
|
||||||
DCHECK(data);
|
DCHECK(data);
|
||||||
|
|
|
@ -26,123 +26,123 @@ uint8_t *plasma_receive(int sock, int64_t message_type);
|
||||||
|
|
||||||
int plasma_send_CreateRequest(int sock,
|
int plasma_send_CreateRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size);
|
int64_t metadata_size);
|
||||||
|
|
||||||
void plasma_read_CreateRequest(uint8_t *data,
|
void plasma_read_CreateRequest(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int64_t *data_size,
|
int64_t *data_size,
|
||||||
int64_t *metadata_size);
|
int64_t *metadata_size);
|
||||||
|
|
||||||
int plasma_send_CreateReply(int sock,
|
int plasma_send_CreateReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
plasma_object *object,
|
PlasmaObject *object,
|
||||||
int error);
|
int error);
|
||||||
|
|
||||||
void plasma_read_CreateReply(uint8_t *data,
|
void plasma_read_CreateReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
plasma_object *object,
|
PlasmaObject *object,
|
||||||
int *error);
|
int *error);
|
||||||
|
|
||||||
/* Plasma Seal message functions. */
|
/* Plasma Seal message functions. */
|
||||||
|
|
||||||
int plasma_send_SealRequest(int sock,
|
int plasma_send_SealRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
unsigned char *digest);
|
unsigned char *digest);
|
||||||
|
|
||||||
void plasma_read_SealRequest(uint8_t *data,
|
void plasma_read_SealRequest(uint8_t *data,
|
||||||
OUT object_id *object_id,
|
OUT ObjectID *object_id,
|
||||||
OUT unsigned char *digest);
|
OUT unsigned char *digest);
|
||||||
|
|
||||||
int plasma_send_SealReply(int sock,
|
int plasma_send_SealReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int error);
|
int error);
|
||||||
|
|
||||||
void plasma_read_SealReply(uint8_t *data, object_id *object_id, int *error);
|
void plasma_read_SealReply(uint8_t *data, ObjectID *object_id, int *error);
|
||||||
|
|
||||||
/* Plasma Get message functions. */
|
/* Plasma Get message functions. */
|
||||||
|
|
||||||
int plasma_send_GetRequest(int sock,
|
int plasma_send_GetRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects,
|
int64_t num_objects,
|
||||||
int64_t timeout_ms);
|
int64_t timeout_ms);
|
||||||
|
|
||||||
int64_t plasma_read_GetRequest_num_objects(uint8_t *data);
|
int64_t plasma_read_GetRequest_num_objects(uint8_t *data);
|
||||||
|
|
||||||
void plasma_read_GetRequest(uint8_t *data,
|
void plasma_read_GetRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t *timeout_ms,
|
int64_t *timeout_ms,
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
int plasma_send_GetReply(int sock,
|
int plasma_send_GetReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
plasma_object plasma_objects[],
|
PlasmaObject plasma_objects[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
void plasma_read_GetReply(uint8_t *data,
|
void plasma_read_GetReply(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
plasma_object plasma_objects[],
|
PlasmaObject plasma_objects[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
/* Plasma Release message functions. */
|
/* Plasma Release message functions. */
|
||||||
|
|
||||||
int plasma_send_ReleaseRequest(int sock,
|
int plasma_send_ReleaseRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
void plasma_read_ReleaseRequest(uint8_t *data, object_id *object_id);
|
void plasma_read_ReleaseRequest(uint8_t *data, ObjectID *object_id);
|
||||||
|
|
||||||
int plasma_send_ReleaseReply(int sock,
|
int plasma_send_ReleaseReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int error);
|
int error);
|
||||||
|
|
||||||
void plasma_read_ReleaseReply(uint8_t *data, object_id *object_id, int *error);
|
void plasma_read_ReleaseReply(uint8_t *data, ObjectID *object_id, int *error);
|
||||||
|
|
||||||
/* Plasma Delete message functions. */
|
/* Plasma Delete message functions. */
|
||||||
|
|
||||||
int plasma_send_DeleteRequest(int sock,
|
int plasma_send_DeleteRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
void plasma_read_DeleteRequest(uint8_t *data, object_id *object_id);
|
void plasma_read_DeleteRequest(uint8_t *data, ObjectID *object_id);
|
||||||
|
|
||||||
int plasma_send_DeleteReply(int sock,
|
int plasma_send_DeleteReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int error);
|
int error);
|
||||||
|
|
||||||
void plasma_read_DeleteReply(uint8_t *data, object_id *object_id, int *error);
|
void plasma_read_DeleteReply(uint8_t *data, ObjectID *object_id, int *error);
|
||||||
|
|
||||||
/* Plasma Status message functions. */
|
/* Plasma Status message functions. */
|
||||||
|
|
||||||
int plasma_send_StatusRequest(int sock,
|
int plasma_send_StatusRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
int64_t plasma_read_StatusRequest_num_objects(uint8_t *data);
|
int64_t plasma_read_StatusRequest_num_objects(uint8_t *data);
|
||||||
|
|
||||||
void plasma_read_StatusRequest(uint8_t *data,
|
void plasma_read_StatusRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
int plasma_send_StatusReply(int sock,
|
int plasma_send_StatusReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int object_status[],
|
int object_status[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
int64_t plasma_read_StatusReply_num_objects(uint8_t *data);
|
int64_t plasma_read_StatusReply_num_objects(uint8_t *data);
|
||||||
|
|
||||||
void plasma_read_StatusReply(uint8_t *data,
|
void plasma_read_StatusReply(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int object_status[],
|
int object_status[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
|
@ -150,17 +150,17 @@ void plasma_read_StatusReply(uint8_t *data,
|
||||||
|
|
||||||
int plasma_send_ContainsRequest(int sock,
|
int plasma_send_ContainsRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id);
|
ObjectID object_id);
|
||||||
|
|
||||||
void plasma_read_ContainsRequest(uint8_t *data, object_id *object_id);
|
void plasma_read_ContainsRequest(uint8_t *data, ObjectID *object_id);
|
||||||
|
|
||||||
int plasma_send_ContainsReply(int sock,
|
int plasma_send_ContainsReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int has_object);
|
int has_object);
|
||||||
|
|
||||||
void plasma_read_ContainsReply(uint8_t *data,
|
void plasma_read_ContainsReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int *has_object);
|
int *has_object);
|
||||||
|
|
||||||
/* Plasma Connect message functions. */
|
/* Plasma Connect message functions. */
|
||||||
|
@ -189,20 +189,20 @@ void plasma_read_EvictReply(uint8_t *data, int64_t *num_bytes);
|
||||||
|
|
||||||
int plasma_send_FetchRequest(int sock,
|
int plasma_send_FetchRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
int64_t plasma_read_FetchRequest_num_objects(uint8_t *data);
|
int64_t plasma_read_FetchRequest_num_objects(uint8_t *data);
|
||||||
|
|
||||||
void plasma_read_FetchRequest(uint8_t *data,
|
void plasma_read_FetchRequest(uint8_t *data,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
int64_t num_objects);
|
int64_t num_objects);
|
||||||
|
|
||||||
/* Plasma Wait message functions. */
|
/* Plasma Wait message functions. */
|
||||||
|
|
||||||
int plasma_send_WaitRequest(int sock,
|
int plasma_send_WaitRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_requests,
|
int num_requests,
|
||||||
int num_ready_objects,
|
int num_ready_objects,
|
||||||
int64_t timeout_ms);
|
int64_t timeout_ms);
|
||||||
|
@ -210,18 +210,18 @@ int plasma_send_WaitRequest(int sock,
|
||||||
int plasma_read_WaitRequest_num_object_ids(uint8_t *data);
|
int plasma_read_WaitRequest_num_object_ids(uint8_t *data);
|
||||||
|
|
||||||
void plasma_read_WaitRequest(uint8_t *data,
|
void plasma_read_WaitRequest(uint8_t *data,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
int64_t *timeout_ms,
|
int64_t *timeout_ms,
|
||||||
int *num_ready_objects);
|
int *num_ready_objects);
|
||||||
|
|
||||||
int plasma_send_WaitReply(int sock,
|
int plasma_send_WaitReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int num_ready_objects);
|
int num_ready_objects);
|
||||||
|
|
||||||
void plasma_read_WaitReply(uint8_t *data,
|
void plasma_read_WaitReply(uint8_t *data,
|
||||||
object_request object_requests[],
|
ObjectRequest object_requests[],
|
||||||
int *num_ready_objects);
|
int *num_ready_objects);
|
||||||
|
|
||||||
/* Plasma Subscribe message functions. */
|
/* Plasma Subscribe message functions. */
|
||||||
|
@ -232,23 +232,23 @@ int plasma_send_SubscribeRequest(int sock, protocol_builder *B);
|
||||||
|
|
||||||
int plasma_send_DataRequest(int sock,
|
int plasma_send_DataRequest(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
const char *address,
|
const char *address,
|
||||||
int port);
|
int port);
|
||||||
|
|
||||||
void plasma_read_DataRequest(uint8_t *data,
|
void plasma_read_DataRequest(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
char **address,
|
char **address,
|
||||||
int *port);
|
int *port);
|
||||||
|
|
||||||
int plasma_send_DataReply(int sock,
|
int plasma_send_DataReply(int sock,
|
||||||
protocol_builder *B,
|
protocol_builder *B,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t object_size,
|
int64_t object_size,
|
||||||
int64_t metadata_size);
|
int64_t metadata_size);
|
||||||
|
|
||||||
void plasma_read_DataReply(uint8_t *data,
|
void plasma_read_DataReply(uint8_t *data,
|
||||||
object_id *object_id,
|
ObjectID *object_id,
|
||||||
int64_t *object_size,
|
int64_t *object_size,
|
||||||
int64_t *metadata_size);
|
int64_t *metadata_size);
|
||||||
|
|
||||||
|
|
|
@ -41,20 +41,20 @@ void *dlmalloc(size_t);
|
||||||
void dlfree(void *);
|
void dlfree(void *);
|
||||||
|
|
||||||
/** Contains all information that is associated with a Plasma store client. */
|
/** Contains all information that is associated with a Plasma store client. */
|
||||||
struct client {
|
struct Client {
|
||||||
/** The socket used to communicate with the client. */
|
/** The socket used to communicate with the client. */
|
||||||
int sock;
|
int sock;
|
||||||
/** A pointer to the global plasma state. */
|
/** A pointer to the global plasma state. */
|
||||||
plasma_store_state *plasma_state;
|
PlasmaStoreState *plasma_state;
|
||||||
};
|
};
|
||||||
|
|
||||||
/* This is used to define the array of clients used to define the
|
/* This is used to define the array of clients used to define the
|
||||||
* object_table_entry type. */
|
* object_table_entry type. */
|
||||||
UT_icd client_icd = {sizeof(client *), NULL, NULL, NULL};
|
UT_icd client_icd = {sizeof(Client *), NULL, NULL, NULL};
|
||||||
|
|
||||||
/* This is used to define the queue of object notifications for plasma
|
/* This is used to define the queue of object notifications for plasma
|
||||||
* subscribers. */
|
* subscribers. */
|
||||||
UT_icd object_info_icd = {sizeof(object_info), NULL, NULL, NULL};
|
UT_icd object_info_icd = {sizeof(ObjectInfo), NULL, NULL, NULL};
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** Client file descriptor. This is used as a key for the hash table. */
|
/** Client file descriptor. This is used as a key for the hash table. */
|
||||||
|
@ -64,57 +64,57 @@ typedef struct {
|
||||||
UT_array *object_notifications;
|
UT_array *object_notifications;
|
||||||
/** Handle for the uthash table. */
|
/** Handle for the uthash table. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} notification_queue;
|
} NotificationQueue;
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The client connection that called get. */
|
/** The client connection that called get. */
|
||||||
client *client;
|
Client *client;
|
||||||
/** The ID of the timer that will time out and cause this wait to return to
|
/** The ID of the timer that will time out and cause this wait to return to
|
||||||
* the client if it hasn't already returned. */
|
* the client if it hasn't already returned. */
|
||||||
int64_t timer;
|
int64_t timer;
|
||||||
/** The number of objects in this get request. */
|
/** The number of objects in this get request. */
|
||||||
int64_t num_object_ids;
|
int64_t num_object_ids;
|
||||||
/** The object IDs involved in this request. This is used in the reply. */
|
/** The object IDs involved in this request. This is used in the reply. */
|
||||||
object_id *object_ids;
|
ObjectID *object_ids;
|
||||||
/** The object information for the objects in this request. This is used in
|
/** The object information for the objects in this request. This is used in
|
||||||
* the reply. */
|
* the reply. */
|
||||||
plasma_object *objects;
|
PlasmaObject *objects;
|
||||||
/** The minimum number of objects to wait for in this request. */
|
/** The minimum number of objects to wait for in this request. */
|
||||||
int64_t num_objects_to_wait_for;
|
int64_t num_objects_to_wait_for;
|
||||||
/** The number of object requests in this wait request that are already
|
/** The number of object requests in this wait request that are already
|
||||||
* satisfied. */
|
* satisfied. */
|
||||||
int64_t num_satisfied;
|
int64_t num_satisfied;
|
||||||
} get_request;
|
} GetRequest;
|
||||||
|
|
||||||
typedef struct {
|
typedef struct {
|
||||||
/** The ID of the object. This is used as a key in a hash table. */
|
/** The ID of the object. This is used as a key in a hash table. */
|
||||||
object_id object_id;
|
ObjectID object_id;
|
||||||
/** An array of the get requests involving this object ID. */
|
/** An array of the get requests involving this object ID. */
|
||||||
UT_array *get_requests;
|
UT_array *get_requests;
|
||||||
/** Handle for the uthash table in the store state that keeps track of the get
|
/** Handle for the uthash table in the store state that keeps track of the get
|
||||||
* requests involving this object ID. */
|
* requests involving this object ID. */
|
||||||
UT_hash_handle hh;
|
UT_hash_handle hh;
|
||||||
} object_get_requests;
|
} ObjectGetRequests;
|
||||||
|
|
||||||
/** This is used to define the utarray of get requests in the
|
/** This is used to define the utarray of get requests in the
|
||||||
* object_get_requests struct. */
|
* ObjectGetRequests struct. */
|
||||||
UT_icd get_request_icd = {sizeof(get_request *), NULL, NULL, NULL};
|
UT_icd get_request_icd = {sizeof(GetRequest *), NULL, NULL, NULL};
|
||||||
|
|
||||||
struct plasma_store_state {
|
struct PlasmaStoreState {
|
||||||
/* Event loop of the plasma store. */
|
/* Event loop of the plasma store. */
|
||||||
event_loop *loop;
|
event_loop *loop;
|
||||||
/** A hash table mapping object IDs to a vector of the get requests that are
|
/** A hash table mapping object IDs to a vector of the get requests that are
|
||||||
* waiting for the object to arrive. */
|
* waiting for the object to arrive. */
|
||||||
object_get_requests *object_get_requests;
|
ObjectGetRequests *object_get_requests;
|
||||||
/** The pending notifications that have not been sent to subscribers because
|
/** The pending notifications that have not been sent to subscribers because
|
||||||
* the socket send buffers were full. This is a hash table from client file
|
* the socket send buffers were full. This is a hash table from client file
|
||||||
* descriptor to an array of object_ids to send to that client. */
|
* descriptor to an array of object_ids to send to that client. */
|
||||||
notification_queue *pending_notifications;
|
NotificationQueue *pending_notifications;
|
||||||
/** The plasma store information, including the object tables, that is exposed
|
/** The plasma store information, including the object tables, that is exposed
|
||||||
* to the eviction policy. */
|
* to the eviction policy. */
|
||||||
plasma_store_info *plasma_store_info;
|
PlasmaStoreInfo *plasma_store_info;
|
||||||
/** The state that is managed by the eviction policy. */
|
/** The state that is managed by the eviction policy. */
|
||||||
eviction_state *eviction_state;
|
EvictionState *eviction_state;
|
||||||
/** Input buffer. This is allocated only once to avoid mallocs for every
|
/** Input buffer. This is allocated only once to avoid mallocs for every
|
||||||
* call to process_message. */
|
* call to process_message. */
|
||||||
UT_array *input_buffer;
|
UT_array *input_buffer;
|
||||||
|
@ -124,32 +124,33 @@ struct plasma_store_state {
|
||||||
|
|
||||||
UT_icd byte_icd = {sizeof(uint8_t), NULL, NULL, NULL};
|
UT_icd byte_icd = {sizeof(uint8_t), NULL, NULL, NULL};
|
||||||
|
|
||||||
plasma_store_state *init_plasma_store(event_loop *loop, int64_t system_memory) {
|
PlasmaStoreState *PlasmaStoreState_init(event_loop *loop,
|
||||||
plasma_store_state *state = malloc(sizeof(plasma_store_state));
|
int64_t system_memory) {
|
||||||
|
PlasmaStoreState *state = malloc(sizeof(PlasmaStoreState));
|
||||||
state->loop = loop;
|
state->loop = loop;
|
||||||
state->object_get_requests = NULL;
|
state->object_get_requests = NULL;
|
||||||
state->pending_notifications = NULL;
|
state->pending_notifications = NULL;
|
||||||
/* Initialize the plasma store info. */
|
/* Initialize the plasma store info. */
|
||||||
state->plasma_store_info = malloc(sizeof(plasma_store_info));
|
state->plasma_store_info = malloc(sizeof(PlasmaStoreInfo));
|
||||||
state->plasma_store_info->objects = NULL;
|
state->plasma_store_info->objects = NULL;
|
||||||
state->plasma_store_info->memory_capacity = system_memory;
|
state->plasma_store_info->memory_capacity = system_memory;
|
||||||
/* Initialize the eviction state. */
|
/* Initialize the eviction state. */
|
||||||
state->eviction_state = make_eviction_state();
|
state->eviction_state = EvictionState_init();
|
||||||
utarray_new(state->input_buffer, &byte_icd);
|
utarray_new(state->input_buffer, &byte_icd);
|
||||||
state->builder = make_protocol_builder();
|
state->builder = make_protocol_builder();
|
||||||
return state;
|
return state;
|
||||||
}
|
}
|
||||||
|
|
||||||
void push_notification(plasma_store_state *state,
|
void push_notification(PlasmaStoreState *state,
|
||||||
object_info *object_notification);
|
ObjectInfo *object_notification);
|
||||||
|
|
||||||
/* If this client is not already using the object, add the client to the
|
/* If this client is not already using the object, add the client to the
|
||||||
* object's list of clients, otherwise do nothing. */
|
* object's list of clients, otherwise do nothing. */
|
||||||
void add_client_to_object_clients(object_table_entry *entry,
|
void add_client_to_object_clients(object_table_entry *entry,
|
||||||
client *client_info) {
|
Client *client_info) {
|
||||||
/* Check if this client is already using the object. */
|
/* Check if this client is already using the object. */
|
||||||
for (int i = 0; i < utarray_len(entry->clients); ++i) {
|
for (int i = 0; i < utarray_len(entry->clients); ++i) {
|
||||||
client **c = (client **) utarray_eltptr(entry->clients, i);
|
Client **c = (Client **) utarray_eltptr(entry->clients, i);
|
||||||
if (*c == client_info) {
|
if (*c == client_info) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -159,11 +160,11 @@ void add_client_to_object_clients(object_table_entry *entry,
|
||||||
if (utarray_len(entry->clients) == 0) {
|
if (utarray_len(entry->clients) == 0) {
|
||||||
/* Tell the eviction policy that this object is being used. */
|
/* Tell the eviction policy that this object is being used. */
|
||||||
int64_t num_objects_to_evict;
|
int64_t num_objects_to_evict;
|
||||||
object_id *objects_to_evict;
|
ObjectID *objects_to_evict;
|
||||||
begin_object_access(client_info->plasma_state->eviction_state,
|
EvictionState_begin_object_access(
|
||||||
client_info->plasma_state->plasma_store_info,
|
client_info->plasma_state->eviction_state,
|
||||||
entry->object_id, &num_objects_to_evict,
|
client_info->plasma_state->plasma_store_info, entry->object_id,
|
||||||
&objects_to_evict);
|
&num_objects_to_evict, &objects_to_evict);
|
||||||
remove_objects(client_info->plasma_state, num_objects_to_evict,
|
remove_objects(client_info->plasma_state, num_objects_to_evict,
|
||||||
objects_to_evict);
|
objects_to_evict);
|
||||||
}
|
}
|
||||||
|
@ -172,13 +173,13 @@ void add_client_to_object_clients(object_table_entry *entry,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Create a new object buffer in the hash table. */
|
/* Create a new object buffer in the hash table. */
|
||||||
int create_object(client *client_context,
|
int create_object(Client *client_context,
|
||||||
object_id obj_id,
|
ObjectID obj_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
plasma_object *result) {
|
PlasmaObject *result) {
|
||||||
LOG_DEBUG("creating object"); /* TODO(pcm): add object_id here */
|
LOG_DEBUG("creating object"); /* TODO(pcm): add ObjectID here */
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
object_table_entry *entry;
|
object_table_entry *entry;
|
||||||
/* TODO(swang): Return these error to the client instead of exiting. */
|
/* TODO(swang): Return these error to the client instead of exiting. */
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &obj_id,
|
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &obj_id,
|
||||||
|
@ -190,8 +191,8 @@ int create_object(client *client_context,
|
||||||
}
|
}
|
||||||
/* Tell the eviction policy how much space we need to create this object. */
|
/* Tell the eviction policy how much space we need to create this object. */
|
||||||
int64_t num_objects_to_evict;
|
int64_t num_objects_to_evict;
|
||||||
object_id *objects_to_evict;
|
ObjectID *objects_to_evict;
|
||||||
bool success = require_space(
|
bool success = EvictionState_require_space(
|
||||||
plasma_state->eviction_state, plasma_state->plasma_store_info,
|
plasma_state->eviction_state, plasma_state->plasma_store_info,
|
||||||
data_size + metadata_size, &num_objects_to_evict, &objects_to_evict);
|
data_size + metadata_size, &num_objects_to_evict, &objects_to_evict);
|
||||||
remove_objects(plasma_state, num_objects_to_evict, objects_to_evict);
|
remove_objects(plasma_state, num_objects_to_evict, objects_to_evict);
|
||||||
|
@ -222,7 +223,7 @@ int create_object(client *client_context,
|
||||||
entry->state = PLASMA_CREATED;
|
entry->state = PLASMA_CREATED;
|
||||||
utarray_new(entry->clients, &client_icd);
|
utarray_new(entry->clients, &client_icd);
|
||||||
HASH_ADD(handle, plasma_state->plasma_store_info->objects, object_id,
|
HASH_ADD(handle, plasma_state->plasma_store_info->objects, object_id,
|
||||||
sizeof(object_id), entry);
|
sizeof(ObjectID), entry);
|
||||||
result->handle.store_fd = fd;
|
result->handle.store_fd = fd;
|
||||||
result->handle.mmap_size = map_size;
|
result->handle.mmap_size = map_size;
|
||||||
result->data_offset = offset;
|
result->data_offset = offset;
|
||||||
|
@ -232,24 +233,24 @@ int create_object(client *client_context,
|
||||||
/* Notify the eviction policy that this object was created. This must be done
|
/* Notify the eviction policy that this object was created. This must be done
|
||||||
* immediately before the call to add_client_to_object_clients so that the
|
* immediately before the call to add_client_to_object_clients so that the
|
||||||
* eviction policy does not have an opportunity to evict the object. */
|
* eviction policy does not have an opportunity to evict the object. */
|
||||||
object_created(plasma_state->eviction_state, plasma_state->plasma_store_info,
|
EvictionState_object_created(plasma_state->eviction_state,
|
||||||
obj_id);
|
plasma_state->plasma_store_info, obj_id);
|
||||||
/* Record that this client is using this object. */
|
/* Record that this client is using this object. */
|
||||||
add_client_to_object_clients(entry, client_context);
|
add_client_to_object_clients(entry, client_context);
|
||||||
return PlasmaError_OK;
|
return PlasmaError_OK;
|
||||||
}
|
}
|
||||||
|
|
||||||
void add_get_request_for_object(plasma_store_state *store_state,
|
void add_get_request_for_object(PlasmaStoreState *store_state,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
get_request *get_req) {
|
GetRequest *get_req) {
|
||||||
object_get_requests *object_get_reqs;
|
ObjectGetRequests *object_get_reqs;
|
||||||
HASH_FIND(hh, store_state->object_get_requests, &object_id, sizeof(object_id),
|
HASH_FIND(hh, store_state->object_get_requests, &object_id, sizeof(object_id),
|
||||||
object_get_reqs);
|
object_get_reqs);
|
||||||
/* If there are currently no get requests involving this object ID, create a
|
/* If there are currently no get requests involving this object ID, create a
|
||||||
* new object_get_requests struct for this object ID and add it to the hash
|
* new ObjectGetRequests struct for this object ID and add it to the hash
|
||||||
* table. */
|
* table. */
|
||||||
if (object_get_reqs == NULL) {
|
if (object_get_reqs == NULL) {
|
||||||
object_get_reqs = malloc(sizeof(object_get_requests));
|
object_get_reqs = malloc(sizeof(ObjectGetRequests));
|
||||||
object_get_reqs->object_id = object_id;
|
object_get_reqs->object_id = object_id;
|
||||||
utarray_new(object_get_reqs->get_requests, &get_request_icd);
|
utarray_new(object_get_reqs->get_requests, &get_request_icd);
|
||||||
HASH_ADD(hh, store_state->object_get_requests, object_id,
|
HASH_ADD(hh, store_state->object_get_requests, object_id,
|
||||||
|
@ -260,18 +261,18 @@ void add_get_request_for_object(plasma_store_state *store_state,
|
||||||
utarray_push_back(object_get_reqs->get_requests, &get_req);
|
utarray_push_back(object_get_reqs->get_requests, &get_req);
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_get_request_for_object(plasma_store_state *store_state,
|
void remove_get_request_for_object(PlasmaStoreState *store_state,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
get_request *get_req) {
|
GetRequest *get_req) {
|
||||||
object_get_requests *object_get_reqs;
|
ObjectGetRequests *object_get_reqs;
|
||||||
HASH_FIND(hh, store_state->object_get_requests, &object_id, sizeof(object_id),
|
HASH_FIND(hh, store_state->object_get_requests, &object_id, sizeof(object_id),
|
||||||
object_get_reqs);
|
object_get_reqs);
|
||||||
/* If there is a vector of get requests for this object ID, and if this vector
|
/* If there is a vector of get requests for this object ID, and if this vector
|
||||||
* contains the get request, then remove the get request from the vector. */
|
* contains the get request, then remove the get request from the vector. */
|
||||||
if (object_get_reqs != NULL) {
|
if (object_get_reqs != NULL) {
|
||||||
for (int i = 0; i < utarray_len(object_get_reqs->get_requests); ++i) {
|
for (int i = 0; i < utarray_len(object_get_reqs->get_requests); ++i) {
|
||||||
get_request **get_req_ptr =
|
GetRequest **get_req_ptr =
|
||||||
(get_request **) utarray_eltptr(object_get_reqs->get_requests, i);
|
(GetRequest **) utarray_eltptr(object_get_reqs->get_requests, i);
|
||||||
if (*get_req_ptr == get_req) {
|
if (*get_req_ptr == get_req) {
|
||||||
/* Remove the get request from the array. */
|
/* Remove the get request from the array. */
|
||||||
utarray_erase(object_get_reqs->get_requests, i, 1);
|
utarray_erase(object_get_reqs->get_requests, i, 1);
|
||||||
|
@ -284,7 +285,7 @@ void remove_get_request_for_object(plasma_store_state *store_state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_get_request(plasma_store_state *store_state, get_request *get_req) {
|
void remove_get_request(PlasmaStoreState *store_state, GetRequest *get_req) {
|
||||||
if (get_req->timer != -1) {
|
if (get_req->timer != -1) {
|
||||||
CHECK(event_loop_remove_timer(store_state->loop, get_req->timer) == AE_OK);
|
CHECK(event_loop_remove_timer(store_state->loop, get_req->timer) == AE_OK);
|
||||||
}
|
}
|
||||||
|
@ -293,8 +294,7 @@ void remove_get_request(plasma_store_state *store_state, get_request *get_req) {
|
||||||
free(get_req);
|
free(get_req);
|
||||||
}
|
}
|
||||||
|
|
||||||
void initialize_plasma_object(plasma_object *object,
|
void PlasmaObject_init(PlasmaObject *object, object_table_entry *entry) {
|
||||||
object_table_entry *entry) {
|
|
||||||
DCHECK(object != NULL);
|
DCHECK(object != NULL);
|
||||||
DCHECK(entry != NULL);
|
DCHECK(entry != NULL);
|
||||||
DCHECK(entry->state == PLASMA_SEALED);
|
DCHECK(entry->state == PLASMA_SEALED);
|
||||||
|
@ -306,7 +306,7 @@ void initialize_plasma_object(plasma_object *object,
|
||||||
object->metadata_size = entry->info.metadata_size;
|
object->metadata_size = entry->info.metadata_size;
|
||||||
}
|
}
|
||||||
|
|
||||||
void return_from_get(plasma_store_state *store_state, get_request *get_req) {
|
void return_from_get(PlasmaStoreState *store_state, GetRequest *get_req) {
|
||||||
/* Send the get reply to the client. */
|
/* Send the get reply to the client. */
|
||||||
int status = plasma_send_GetReply(get_req->client->sock, store_state->builder,
|
int status = plasma_send_GetReply(get_req->client->sock, store_state->builder,
|
||||||
get_req->object_ids, get_req->objects,
|
get_req->object_ids, get_req->objects,
|
||||||
|
@ -352,10 +352,10 @@ void return_from_get(plasma_store_state *store_state, get_request *get_req) {
|
||||||
remove_get_request(store_state, get_req);
|
remove_get_request(store_state, get_req);
|
||||||
}
|
}
|
||||||
|
|
||||||
void update_object_get_requests(plasma_store_state *store_state,
|
void update_object_get_requests(PlasmaStoreState *store_state,
|
||||||
object_id obj_id) {
|
ObjectID obj_id) {
|
||||||
/* Update the in-progress get requests. */
|
/* Update the in-progress get requests. */
|
||||||
object_get_requests *object_get_reqs;
|
ObjectGetRequests *object_get_reqs;
|
||||||
HASH_FIND(hh, store_state->object_get_requests, &obj_id, sizeof(obj_id),
|
HASH_FIND(hh, store_state->object_get_requests, &obj_id, sizeof(obj_id),
|
||||||
object_get_reqs);
|
object_get_reqs);
|
||||||
if (object_get_reqs != NULL) {
|
if (object_get_reqs != NULL) {
|
||||||
|
@ -368,9 +368,9 @@ void update_object_get_requests(plasma_store_state *store_state,
|
||||||
* are removed from the array. */
|
* are removed from the array. */
|
||||||
int index = 0;
|
int index = 0;
|
||||||
for (int i = 0; i < num_requests; ++i) {
|
for (int i = 0; i < num_requests; ++i) {
|
||||||
get_request **get_req_ptr =
|
GetRequest **get_req_ptr =
|
||||||
(get_request **) utarray_eltptr(object_get_reqs->get_requests, index);
|
(GetRequest **) utarray_eltptr(object_get_reqs->get_requests, index);
|
||||||
get_request *get_req = *get_req_ptr;
|
GetRequest *get_req = *get_req_ptr;
|
||||||
|
|
||||||
int num_updated = 0;
|
int num_updated = 0;
|
||||||
for (int j = 0; j < get_req->num_objects_to_wait_for; ++j) {
|
for (int j = 0; j < get_req->num_objects_to_wait_for; ++j) {
|
||||||
|
@ -379,8 +379,8 @@ void update_object_get_requests(plasma_store_state *store_state,
|
||||||
sizeof(obj_id), entry);
|
sizeof(obj_id), entry);
|
||||||
CHECK(entry != NULL);
|
CHECK(entry != NULL);
|
||||||
|
|
||||||
if (object_ids_equal(get_req->object_ids[j], obj_id)) {
|
if (ObjectID_equal(get_req->object_ids[j], obj_id)) {
|
||||||
initialize_plasma_object(&get_req->objects[j], entry);
|
PlasmaObject_init(&get_req->objects[j], entry);
|
||||||
num_updated += 1;
|
num_updated += 1;
|
||||||
get_req->num_satisfied += 1;
|
get_req->num_satisfied += 1;
|
||||||
/* Record the fact that this client will be using this object and will
|
/* Record the fact that this client will be using this object and will
|
||||||
|
@ -413,25 +413,25 @@ void update_object_get_requests(plasma_store_state *store_state,
|
||||||
}
|
}
|
||||||
|
|
||||||
int get_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
int get_timeout_handler(event_loop *loop, timer_id id, void *context) {
|
||||||
get_request *get_req = context;
|
GetRequest *get_req = context;
|
||||||
return_from_get(get_req->client->plasma_state, get_req);
|
return_from_get(get_req->client->plasma_state, get_req);
|
||||||
return EVENT_LOOP_TIMER_DONE;
|
return EVENT_LOOP_TIMER_DONE;
|
||||||
}
|
}
|
||||||
|
|
||||||
void process_get_request(client *client_context,
|
void process_get_request(Client *client_context,
|
||||||
int num_object_ids,
|
int num_object_ids,
|
||||||
object_id object_ids[],
|
ObjectID object_ids[],
|
||||||
uint64_t timeout_ms) {
|
uint64_t timeout_ms) {
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
|
|
||||||
/* Create a get request for this object. */
|
/* Create a get request for this object. */
|
||||||
get_request *get_req = malloc(sizeof(get_request));
|
GetRequest *get_req = malloc(sizeof(GetRequest));
|
||||||
memset(get_req, 0, sizeof(get_request));
|
memset(get_req, 0, sizeof(GetRequest));
|
||||||
get_req->client = client_context;
|
get_req->client = client_context;
|
||||||
get_req->timer = -1;
|
get_req->timer = -1;
|
||||||
get_req->num_object_ids = num_object_ids;
|
get_req->num_object_ids = num_object_ids;
|
||||||
get_req->object_ids = malloc(num_object_ids * sizeof(object_id));
|
get_req->object_ids = malloc(num_object_ids * sizeof(ObjectID));
|
||||||
get_req->objects = malloc(num_object_ids * sizeof(plasma_object));
|
get_req->objects = malloc(num_object_ids * sizeof(PlasmaObject));
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
for (int i = 0; i < num_object_ids; ++i) {
|
||||||
get_req->object_ids[i] = object_ids[i];
|
get_req->object_ids[i] = object_ids[i];
|
||||||
}
|
}
|
||||||
|
@ -439,7 +439,7 @@ void process_get_request(client *client_context,
|
||||||
get_req->num_satisfied = 0;
|
get_req->num_satisfied = 0;
|
||||||
|
|
||||||
for (int i = 0; i < num_object_ids; ++i) {
|
for (int i = 0; i < num_object_ids; ++i) {
|
||||||
object_id obj_id = object_ids[i];
|
ObjectID obj_id = object_ids[i];
|
||||||
|
|
||||||
/* Check if this object is already present locally. If so, record that the
|
/* Check if this object is already present locally. If so, record that the
|
||||||
* object is being used and mark it as accounted for. */
|
* object is being used and mark it as accounted for. */
|
||||||
|
@ -448,7 +448,7 @@ void process_get_request(client *client_context,
|
||||||
sizeof(obj_id), entry);
|
sizeof(obj_id), entry);
|
||||||
if (entry && entry->state == PLASMA_SEALED) {
|
if (entry && entry->state == PLASMA_SEALED) {
|
||||||
/* Update the get request to take into account the present object. */
|
/* Update the get request to take into account the present object. */
|
||||||
initialize_plasma_object(&get_req->objects[i], entry);
|
PlasmaObject_init(&get_req->objects[i], entry);
|
||||||
get_req->num_satisfied += 1;
|
get_req->num_satisfied += 1;
|
||||||
/* If necessary, record that this client is using this object. In the case
|
/* If necessary, record that this client is using this object. In the case
|
||||||
* where entry == NULL, this will be called from seal_object. */
|
* where entry == NULL, this will be called from seal_object. */
|
||||||
|
@ -478,35 +478,11 @@ void process_get_request(client *client_context,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Get an object from the local Plasma Store if exists. */
|
|
||||||
int get_object_local(client *client_context,
|
|
||||||
int conn,
|
|
||||||
object_id object_id,
|
|
||||||
plasma_object *result) {
|
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
|
||||||
object_table_entry *entry;
|
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
|
||||||
sizeof(object_id), entry);
|
|
||||||
if (entry && entry->state == PLASMA_SEALED) {
|
|
||||||
result->handle.store_fd = entry->fd;
|
|
||||||
result->handle.mmap_size = entry->map_size;
|
|
||||||
result->data_offset = entry->offset;
|
|
||||||
result->metadata_offset = entry->offset + entry->info.data_size;
|
|
||||||
result->data_size = entry->info.data_size;
|
|
||||||
result->metadata_size = entry->info.metadata_size;
|
|
||||||
/* If necessary, record that this client is using this object. In the case
|
|
||||||
* where entry == NULL, this will be called from seal_object. */
|
|
||||||
add_client_to_object_clients(entry, client_context);
|
|
||||||
return OBJECT_FOUND;
|
|
||||||
}
|
|
||||||
return OBJECT_NOT_FOUND;
|
|
||||||
}
|
|
||||||
|
|
||||||
int remove_client_from_object_clients(object_table_entry *entry,
|
int remove_client_from_object_clients(object_table_entry *entry,
|
||||||
client *client_info) {
|
Client *client_info) {
|
||||||
/* Find the location of the client in the array. */
|
/* Find the location of the client in the array. */
|
||||||
for (int i = 0; i < utarray_len(entry->clients); ++i) {
|
for (int i = 0; i < utarray_len(entry->clients); ++i) {
|
||||||
client **c = (client **) utarray_eltptr(entry->clients, i);
|
Client **c = (Client **) utarray_eltptr(entry->clients, i);
|
||||||
if (*c == client_info) {
|
if (*c == client_info) {
|
||||||
/* Remove the client from the array. */
|
/* Remove the client from the array. */
|
||||||
utarray_erase(entry->clients, i, 1);
|
utarray_erase(entry->clients, i, 1);
|
||||||
|
@ -515,11 +491,11 @@ int remove_client_from_object_clients(object_table_entry *entry,
|
||||||
if (utarray_len(entry->clients) == 0) {
|
if (utarray_len(entry->clients) == 0) {
|
||||||
/* Tell the eviction policy that this object is no longer being used. */
|
/* Tell the eviction policy that this object is no longer being used. */
|
||||||
int64_t num_objects_to_evict;
|
int64_t num_objects_to_evict;
|
||||||
object_id *objects_to_evict;
|
ObjectID *objects_to_evict;
|
||||||
end_object_access(client_info->plasma_state->eviction_state,
|
EvictionState_end_object_access(
|
||||||
client_info->plasma_state->plasma_store_info,
|
client_info->plasma_state->eviction_state,
|
||||||
entry->object_id, &num_objects_to_evict,
|
client_info->plasma_state->plasma_store_info, entry->object_id,
|
||||||
&objects_to_evict);
|
&num_objects_to_evict, &objects_to_evict);
|
||||||
remove_objects(client_info->plasma_state, num_objects_to_evict,
|
remove_objects(client_info->plasma_state, num_objects_to_evict,
|
||||||
objects_to_evict);
|
objects_to_evict);
|
||||||
}
|
}
|
||||||
|
@ -531,8 +507,8 @@ int remove_client_from_object_clients(object_table_entry *entry,
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
void release_object(client *client_context, object_id object_id) {
|
void release_object(Client *client_context, ObjectID object_id) {
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
object_table_entry *entry;
|
object_table_entry *entry;
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
||||||
sizeof(object_id), entry);
|
sizeof(object_id), entry);
|
||||||
|
@ -542,8 +518,8 @@ void release_object(client *client_context, object_id object_id) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Check if an object is present. */
|
/* Check if an object is present. */
|
||||||
int contains_object(client *client_context, object_id object_id) {
|
int contains_object(Client *client_context, ObjectID object_id) {
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
object_table_entry *entry;
|
object_table_entry *entry;
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
||||||
sizeof(object_id), entry);
|
sizeof(object_id), entry);
|
||||||
|
@ -552,11 +528,11 @@ int contains_object(client *client_context, object_id object_id) {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Seal an object that has been created in the hash table. */
|
/* Seal an object that has been created in the hash table. */
|
||||||
void seal_object(client *client_context,
|
void seal_object(Client *client_context,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
unsigned char digest[]) {
|
unsigned char digest[]) {
|
||||||
LOG_DEBUG("sealing object"); // TODO(pcm): add object_id here
|
LOG_DEBUG("sealing object"); // TODO(pcm): add ObjectID here
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
object_table_entry *entry;
|
object_table_entry *entry;
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
||||||
sizeof(object_id), entry);
|
sizeof(object_id), entry);
|
||||||
|
@ -575,7 +551,7 @@ void seal_object(client *client_context,
|
||||||
|
|
||||||
/* Delete an object that has been created in the hash table. This should only
|
/* Delete an object that has been created in the hash table. This should only
|
||||||
* be called on objects that are returned by the eviction policy to evict. */
|
* be called on objects that are returned by the eviction policy to evict. */
|
||||||
void delete_object(plasma_store_state *plasma_state, object_id object_id) {
|
void delete_object(PlasmaStoreState *plasma_state, ObjectID object_id) {
|
||||||
LOG_DEBUG("deleting object");
|
LOG_DEBUG("deleting object");
|
||||||
object_table_entry *entry;
|
object_table_entry *entry;
|
||||||
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
HASH_FIND(handle, plasma_state->plasma_store_info->objects, &object_id,
|
||||||
|
@ -594,13 +570,13 @@ void delete_object(plasma_store_state *plasma_state, object_id object_id) {
|
||||||
utarray_free(entry->clients);
|
utarray_free(entry->clients);
|
||||||
free(entry);
|
free(entry);
|
||||||
/* Inform all subscribers that the object has been deleted. */
|
/* Inform all subscribers that the object has been deleted. */
|
||||||
object_info notification = {.obj_id = object_id, .is_deletion = true};
|
ObjectInfo notification = {.obj_id = object_id, .is_deletion = true};
|
||||||
push_notification(plasma_state, ¬ification);
|
push_notification(plasma_state, ¬ification);
|
||||||
}
|
}
|
||||||
|
|
||||||
void remove_objects(plasma_store_state *plasma_state,
|
void remove_objects(PlasmaStoreState *plasma_state,
|
||||||
int64_t num_objects_to_evict,
|
int64_t num_objects_to_evict,
|
||||||
object_id *objects_to_evict) {
|
ObjectID *objects_to_evict) {
|
||||||
if (num_objects_to_evict > 0) {
|
if (num_objects_to_evict > 0) {
|
||||||
for (int i = 0; i < num_objects_to_evict; ++i) {
|
for (int i = 0; i < num_objects_to_evict; ++i) {
|
||||||
delete_object(plasma_state, objects_to_evict[i]);
|
delete_object(plasma_state, objects_to_evict[i]);
|
||||||
|
@ -611,9 +587,9 @@ void remove_objects(plasma_store_state *plasma_state,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void push_notification(plasma_store_state *plasma_state,
|
void push_notification(PlasmaStoreState *plasma_state,
|
||||||
object_info *notification) {
|
ObjectInfo *notification) {
|
||||||
notification_queue *queue, *temp_queue;
|
NotificationQueue *queue, *temp_queue;
|
||||||
HASH_ITER(hh, plasma_state->pending_notifications, queue, temp_queue) {
|
HASH_ITER(hh, plasma_state->pending_notifications, queue, temp_queue) {
|
||||||
utarray_push_back(queue->object_notifications, notification);
|
utarray_push_back(queue->object_notifications, notification);
|
||||||
send_notifications(plasma_state->loop, queue->subscriber_fd, plasma_state,
|
send_notifications(plasma_state->loop, queue->subscriber_fd, plasma_state,
|
||||||
|
@ -626,8 +602,8 @@ void send_notifications(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
plasma_store_state *plasma_state = context;
|
PlasmaStoreState *plasma_state = context;
|
||||||
notification_queue *queue;
|
NotificationQueue *queue;
|
||||||
HASH_FIND_INT(plasma_state->pending_notifications, &client_sock, queue);
|
HASH_FIND_INT(plasma_state->pending_notifications, &client_sock, queue);
|
||||||
CHECK(queue != NULL);
|
CHECK(queue != NULL);
|
||||||
|
|
||||||
|
@ -635,14 +611,14 @@ void send_notifications(event_loop *loop,
|
||||||
/* Loop over the array of pending notifications and send as many of them as
|
/* Loop over the array of pending notifications and send as many of them as
|
||||||
* possible. */
|
* possible. */
|
||||||
for (int i = 0; i < utarray_len(queue->object_notifications); ++i) {
|
for (int i = 0; i < utarray_len(queue->object_notifications); ++i) {
|
||||||
object_info *notification =
|
ObjectInfo *notification =
|
||||||
(object_info *) utarray_eltptr(queue->object_notifications, i);
|
(ObjectInfo *) utarray_eltptr(queue->object_notifications, i);
|
||||||
|
|
||||||
/* Attempt to send a notification about this object ID. */
|
/* Attempt to send a notification about this object ID. */
|
||||||
int nbytes =
|
int nbytes = send(client_sock, (char const *) notification,
|
||||||
send(client_sock, (char const *) notification, sizeof(object_info), 0);
|
sizeof(*notification), 0);
|
||||||
if (nbytes >= 0) {
|
if (nbytes >= 0) {
|
||||||
CHECK(nbytes == sizeof(object_info));
|
CHECK(nbytes == sizeof(*notification));
|
||||||
} else if (nbytes == -1 &&
|
} else if (nbytes == -1 &&
|
||||||
(errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR)) {
|
(errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR)) {
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
|
@ -669,9 +645,9 @@ void send_notifications(event_loop *loop,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Subscribe to notifications about sealed objects. */
|
/* Subscribe to notifications about sealed objects. */
|
||||||
void subscribe_to_updates(client *client_context, int conn) {
|
void subscribe_to_updates(Client *client_context, int conn) {
|
||||||
LOG_DEBUG("subscribing to updates");
|
LOG_DEBUG("subscribing to updates");
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
/* TODO(rkn): The store could block here if the client doesn't send a file
|
/* TODO(rkn): The store could block here if the client doesn't send a file
|
||||||
* descriptor. */
|
* descriptor. */
|
||||||
int fd = recv_fd(conn);
|
int fd = recv_fd(conn);
|
||||||
|
@ -684,8 +660,8 @@ void subscribe_to_updates(client *client_context, int conn) {
|
||||||
/* Create a new array to buffer notifications that can't be sent to the
|
/* Create a new array to buffer notifications that can't be sent to the
|
||||||
* subscriber yet because the socket send buffer is full. TODO(rkn): the queue
|
* subscriber yet because the socket send buffer is full. TODO(rkn): the queue
|
||||||
* never gets freed. */
|
* never gets freed. */
|
||||||
notification_queue *queue =
|
NotificationQueue *queue =
|
||||||
(notification_queue *) malloc(sizeof(notification_queue));
|
(NotificationQueue *) malloc(sizeof(NotificationQueue));
|
||||||
queue->subscriber_fd = fd;
|
queue->subscriber_fd = fd;
|
||||||
utarray_new(queue->object_notifications, &object_info_icd);
|
utarray_new(queue->object_notifications, &object_info_icd);
|
||||||
HASH_ADD_INT(plasma_state->pending_notifications, subscriber_fd, queue);
|
HASH_ADD_INT(plasma_state->pending_notifications, subscriber_fd, queue);
|
||||||
|
@ -703,15 +679,15 @@ void process_message(event_loop *loop,
|
||||||
int client_sock,
|
int client_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
client *client_context = context;
|
Client *client_context = context;
|
||||||
plasma_store_state *state = client_context->plasma_state;
|
PlasmaStoreState *state = client_context->plasma_state;
|
||||||
int64_t type;
|
int64_t type;
|
||||||
read_buffer(client_sock, &type, state->input_buffer);
|
read_buffer(client_sock, &type, state->input_buffer);
|
||||||
|
|
||||||
uint8_t *input = (uint8_t *) utarray_front(state->input_buffer);
|
uint8_t *input = (uint8_t *) utarray_front(state->input_buffer);
|
||||||
object_id object_ids[1];
|
ObjectID object_ids[1];
|
||||||
int64_t num_objects;
|
int64_t num_objects;
|
||||||
plasma_object objects[1];
|
PlasmaObject objects[1];
|
||||||
memset(&objects[0], 0, sizeof(objects));
|
memset(&objects[0], 0, sizeof(objects));
|
||||||
int error;
|
int error;
|
||||||
|
|
||||||
|
@ -737,7 +713,7 @@ void process_message(event_loop *loop,
|
||||||
} break;
|
} break;
|
||||||
case MessageType_PlasmaGetRequest: {
|
case MessageType_PlasmaGetRequest: {
|
||||||
num_objects = plasma_read_GetRequest_num_objects(input);
|
num_objects = plasma_read_GetRequest_num_objects(input);
|
||||||
object_id *object_ids_to_get = malloc(num_objects * sizeof(object_id));
|
ObjectID *object_ids_to_get = malloc(num_objects * sizeof(ObjectID));
|
||||||
int64_t timeout_ms;
|
int64_t timeout_ms;
|
||||||
plasma_read_GetRequest(input, object_ids_to_get, &timeout_ms, num_objects);
|
plasma_read_GetRequest(input, object_ids_to_get, &timeout_ms, num_objects);
|
||||||
/* TODO(pcm): The array object_ids_to_get could be reused in
|
/* TODO(pcm): The array object_ids_to_get could be reused in
|
||||||
|
@ -772,8 +748,8 @@ void process_message(event_loop *loop,
|
||||||
int64_t num_bytes;
|
int64_t num_bytes;
|
||||||
plasma_read_EvictRequest(input, &num_bytes);
|
plasma_read_EvictRequest(input, &num_bytes);
|
||||||
int64_t num_objects_to_evict;
|
int64_t num_objects_to_evict;
|
||||||
object_id *objects_to_evict;
|
ObjectID *objects_to_evict;
|
||||||
int64_t num_bytes_evicted = choose_objects_to_evict(
|
int64_t num_bytes_evicted = EvictionState_choose_objects_to_evict(
|
||||||
client_context->plasma_state->eviction_state,
|
client_context->plasma_state->eviction_state,
|
||||||
client_context->plasma_state->plasma_store_info, num_bytes,
|
client_context->plasma_state->plasma_store_info, num_bytes,
|
||||||
&num_objects_to_evict, &objects_to_evict);
|
&num_objects_to_evict, &objects_to_evict);
|
||||||
|
@ -797,7 +773,7 @@ void process_message(event_loop *loop,
|
||||||
event_loop_remove_file(loop, client_sock);
|
event_loop_remove_file(loop, client_sock);
|
||||||
/* If this client was using any objects, remove it from the appropriate
|
/* If this client was using any objects, remove it from the appropriate
|
||||||
* lists. */
|
* lists. */
|
||||||
plasma_store_state *plasma_state = client_context->plasma_state;
|
PlasmaStoreState *plasma_state = client_context->plasma_state;
|
||||||
object_table_entry *entry, *temp_entry;
|
object_table_entry *entry, *temp_entry;
|
||||||
HASH_ITER(handle, plasma_state->plasma_store_info->objects, entry,
|
HASH_ITER(handle, plasma_state->plasma_store_info->objects, entry,
|
||||||
temp_entry) {
|
temp_entry) {
|
||||||
|
@ -818,11 +794,11 @@ void new_client_connection(event_loop *loop,
|
||||||
int listener_sock,
|
int listener_sock,
|
||||||
void *context,
|
void *context,
|
||||||
int events) {
|
int events) {
|
||||||
plasma_store_state *plasma_state = context;
|
PlasmaStoreState *plasma_state = context;
|
||||||
int new_socket = accept_client(listener_sock);
|
int new_socket = accept_client(listener_sock);
|
||||||
/* Create a new client object. This will also be used as the context to use
|
/* Create a new client object. This will also be used as the context to use
|
||||||
* for events on this client's socket. TODO(rkn): free this somewhere. */
|
* for events on this client's socket. TODO(rkn): free this somewhere. */
|
||||||
client *client_context = (client *) malloc(sizeof(client));
|
Client *client_context = (Client *) malloc(sizeof(Client));
|
||||||
client_context->sock = new_socket;
|
client_context->sock = new_socket;
|
||||||
client_context->plasma_state = plasma_state;
|
client_context->plasma_state = plasma_state;
|
||||||
/* Add a callback to handle events on this socket. */
|
/* Add a callback to handle events on this socket. */
|
||||||
|
@ -844,7 +820,7 @@ void start_server(char *socket_name, int64_t system_memory) {
|
||||||
signal(SIGPIPE, SIG_IGN);
|
signal(SIGPIPE, SIG_IGN);
|
||||||
/* Create the event loop. */
|
/* Create the event loop. */
|
||||||
event_loop *loop = event_loop_create();
|
event_loop *loop = event_loop_create();
|
||||||
plasma_store_state *state = init_plasma_store(loop, system_memory);
|
PlasmaStoreState *state = PlasmaStoreState_init(loop, system_memory);
|
||||||
int socket = bind_ipc_sock(socket_name, true);
|
int socket = bind_ipc_sock(socket_name, true);
|
||||||
CHECK(socket >= 0);
|
CHECK(socket >= 0);
|
||||||
event_loop_add_file(loop, socket, EVENT_LOOP_READ, new_client_connection,
|
event_loop_add_file(loop, socket, EVENT_LOOP_READ, new_client_connection,
|
||||||
|
|
|
@ -3,9 +3,9 @@
|
||||||
|
|
||||||
#include "plasma.h"
|
#include "plasma.h"
|
||||||
|
|
||||||
typedef struct client client;
|
typedef struct Client Client;
|
||||||
|
|
||||||
typedef struct plasma_store_state plasma_store_state;
|
typedef struct PlasmaStoreState PlasmaStoreState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new object. The client must do a call to release_object to tell the
|
* Create a new object. The client must do a call to release_object to tell the
|
||||||
|
@ -24,11 +24,11 @@ typedef struct plasma_store_state plasma_store_state;
|
||||||
* create the object. In this case, the client should not call
|
* create the object. In this case, the client should not call
|
||||||
* plasma_release.
|
* plasma_release.
|
||||||
*/
|
*/
|
||||||
int create_object(client *client_context,
|
int create_object(Client *client_context,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
int64_t data_size,
|
int64_t data_size,
|
||||||
int64_t metadata_size,
|
int64_t metadata_size,
|
||||||
plasma_object *result);
|
PlasmaObject *result);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get an object. This method assumes that we currently have or will eventually
|
* Get an object. This method assumes that we currently have or will eventually
|
||||||
|
@ -43,28 +43,10 @@ int create_object(client *client_context,
|
||||||
* @param object_id Object ID of the object to be gotten.
|
* @param object_id Object ID of the object to be gotten.
|
||||||
* @return The status of the object (object_status in plasma.h).
|
* @return The status of the object (object_status in plasma.h).
|
||||||
*/
|
*/
|
||||||
int get_object(client *client_context,
|
int get_object(Client *client_context,
|
||||||
int conn,
|
int conn,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
plasma_object *result);
|
PlasmaObject *result);
|
||||||
|
|
||||||
/**
|
|
||||||
* Get an object from the local Plasma Store. This function is not blocking.
|
|
||||||
*
|
|
||||||
* Once a client gets an object it must release it when it is done with it.
|
|
||||||
* This function is indepontent. If a client calls repeatedly get_object_local()
|
|
||||||
* on the same object_id, the client needs to call release_object() only once.
|
|
||||||
*
|
|
||||||
* @param client_context The context of the client making this request.
|
|
||||||
* @param conn The client connection that requests the object.
|
|
||||||
* @param object_id Object ID of the object to be gotten.
|
|
||||||
* @return Return OBJECT_FOUND if object was found, and OBJECT_NOT_FOUND
|
|
||||||
* otherwise.
|
|
||||||
*/
|
|
||||||
int get_object_local(client *client_context,
|
|
||||||
int conn,
|
|
||||||
object_id object_id,
|
|
||||||
plasma_object *result);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Record the fact that a particular client is no longer using an object.
|
* Record the fact that a particular client is no longer using an object.
|
||||||
|
@ -73,7 +55,7 @@ int get_object_local(client *client_context,
|
||||||
* @param object_id The object ID of the object that is being released.
|
* @param object_id The object ID of the object that is being released.
|
||||||
* @param Void.
|
* @param Void.
|
||||||
*/
|
*/
|
||||||
void release_object(client *client_context, object_id object_id);
|
void release_object(Client *client_context, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Seal an object. The object is now immutable and can be accessed with get.
|
* Seal an object. The object is now immutable and can be accessed with get.
|
||||||
|
@ -84,8 +66,8 @@ void release_object(client *client_context, object_id object_id);
|
||||||
* with the same object ID are the same.
|
* with the same object ID are the same.
|
||||||
* @return Void.
|
* @return Void.
|
||||||
*/
|
*/
|
||||||
void seal_object(client *client_context,
|
void seal_object(Client *client_context,
|
||||||
object_id object_id,
|
ObjectID object_id,
|
||||||
unsigned char digest[]);
|
unsigned char digest[]);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -95,7 +77,7 @@ void seal_object(client *client_context,
|
||||||
* @param object_id Object ID that will be checked.
|
* @param object_id Object ID that will be checked.
|
||||||
* @return OBJECT_FOUND if the object is in the store, OBJECT_NOT_FOUND if not
|
* @return OBJECT_FOUND if the object is in the store, OBJECT_NOT_FOUND if not
|
||||||
*/
|
*/
|
||||||
int contains_object(client *client_context, object_id object_id);
|
int contains_object(Client *client_context, ObjectID object_id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Send notifications about sealed objects to the subscribers. This is called
|
* Send notifications about sealed objects to the subscribers. This is called
|
||||||
|
@ -114,8 +96,8 @@ void send_notifications(event_loop *loop,
|
||||||
void *plasma_state,
|
void *plasma_state,
|
||||||
int events);
|
int events);
|
||||||
|
|
||||||
void remove_objects(plasma_store_state *plasma_state,
|
void remove_objects(PlasmaStoreState *plasma_state,
|
||||||
int64_t num_objects_to_evict,
|
int64_t num_objects_to_evict,
|
||||||
object_id *objects_to_evict);
|
ObjectID *objects_to_evict);
|
||||||
|
|
||||||
#endif /* PLASMA_STORE_H */
|
#endif /* PLASMA_STORE_H */
|
||||||
|
|
|
@ -11,11 +11,11 @@
|
||||||
SUITE(plasma_client_tests);
|
SUITE(plasma_client_tests);
|
||||||
|
|
||||||
TEST plasma_status_tests(void) {
|
TEST plasma_status_tests(void) {
|
||||||
plasma_connection *plasma_conn1 = plasma_connect(
|
PlasmaConnection *plasma_conn1 = plasma_connect(
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
plasma_connection *plasma_conn2 = plasma_connect(
|
PlasmaConnection *plasma_conn2 = plasma_connect(
|
||||||
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid1 = globally_unique_id();
|
ObjectID oid1 = globally_unique_id();
|
||||||
|
|
||||||
/* Test for object non-existence. */
|
/* Test for object non-existence. */
|
||||||
int status = plasma_status(plasma_conn1, oid1);
|
int status = plasma_status(plasma_conn1, oid1);
|
||||||
|
@ -46,11 +46,11 @@ TEST plasma_status_tests(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST plasma_fetch_tests(void) {
|
TEST plasma_fetch_tests(void) {
|
||||||
plasma_connection *plasma_conn1 = plasma_connect(
|
PlasmaConnection *plasma_conn1 = plasma_connect(
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
plasma_connection *plasma_conn2 = plasma_connect(
|
PlasmaConnection *plasma_conn2 = plasma_connect(
|
||||||
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid1 = globally_unique_id();
|
ObjectID oid1 = globally_unique_id();
|
||||||
|
|
||||||
/* Test for object non-existence. */
|
/* Test for object non-existence. */
|
||||||
int status;
|
int status;
|
||||||
|
@ -71,7 +71,7 @@ TEST plasma_fetch_tests(void) {
|
||||||
/* Object with ID oid1 has been just inserted. On the next fetch we might
|
/* Object with ID oid1 has been just inserted. On the next fetch we might
|
||||||
* either find the object or not, depending on whether the Plasma Manager has
|
* either find the object or not, depending on whether the Plasma Manager has
|
||||||
* received the notification from the Plasma Store or not. */
|
* received the notification from the Plasma Store or not. */
|
||||||
object_id oid_array1[1] = {oid1};
|
ObjectID oid_array1[1] = {oid1};
|
||||||
plasma_fetch(plasma_conn1, 1, oid_array1);
|
plasma_fetch(plasma_conn1, 1, oid_array1);
|
||||||
status = plasma_status(plasma_conn1, oid1);
|
status = plasma_status(plasma_conn1, oid1);
|
||||||
ASSERT((status == ObjectStatus_Local) ||
|
ASSERT((status == ObjectStatus_Local) ||
|
||||||
|
@ -116,11 +116,11 @@ bool is_equal_data_123(uint8_t *data1, uint8_t *data2, uint64_t size) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST plasma_nonblocking_get_tests(void) {
|
TEST plasma_nonblocking_get_tests(void) {
|
||||||
plasma_connection *plasma_conn = plasma_connect(
|
PlasmaConnection *plasma_conn = plasma_connect("/tmp/store1", "/tmp/manager1",
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid = globally_unique_id();
|
ObjectID oid = globally_unique_id();
|
||||||
object_id oid_array[1] = {oid};
|
ObjectID oid_array[1] = {oid};
|
||||||
object_buffer obj_buffer;
|
ObjectBuffer obj_buffer;
|
||||||
|
|
||||||
/* Test for object non-existence. */
|
/* Test for object non-existence. */
|
||||||
plasma_get(plasma_conn, oid_array, 1, 0, &obj_buffer);
|
plasma_get(plasma_conn, oid_array, 1, 0, &obj_buffer);
|
||||||
|
@ -147,15 +147,15 @@ TEST plasma_nonblocking_get_tests(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST plasma_wait_for_objects_tests(void) {
|
TEST plasma_wait_for_objects_tests(void) {
|
||||||
plasma_connection *plasma_conn1 = plasma_connect(
|
PlasmaConnection *plasma_conn1 = plasma_connect(
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
plasma_connection *plasma_conn2 = plasma_connect(
|
PlasmaConnection *plasma_conn2 = plasma_connect(
|
||||||
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid1 = globally_unique_id();
|
ObjectID oid1 = globally_unique_id();
|
||||||
object_id oid2 = globally_unique_id();
|
ObjectID oid2 = globally_unique_id();
|
||||||
#define NUM_OBJ_REQUEST 2
|
#define NUM_OBJ_REQUEST 2
|
||||||
#define WAIT_TIMEOUT_MS 1000
|
#define WAIT_TIMEOUT_MS 1000
|
||||||
object_request obj_requests[NUM_OBJ_REQUEST];
|
ObjectRequest obj_requests[NUM_OBJ_REQUEST];
|
||||||
|
|
||||||
obj_requests[0].object_id = oid1;
|
obj_requests[0].object_id = oid1;
|
||||||
obj_requests[0].type = PLASMA_QUERY_ANYWHERE;
|
obj_requests[0].type = PLASMA_QUERY_ANYWHERE;
|
||||||
|
@ -214,16 +214,16 @@ TEST plasma_wait_for_objects_tests(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST plasma_get_tests(void) {
|
TEST plasma_get_tests(void) {
|
||||||
plasma_connection *plasma_conn1 = plasma_connect(
|
PlasmaConnection *plasma_conn1 = plasma_connect(
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
plasma_connection *plasma_conn2 = plasma_connect(
|
PlasmaConnection *plasma_conn2 = plasma_connect(
|
||||||
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid1 = globally_unique_id();
|
ObjectID oid1 = globally_unique_id();
|
||||||
object_id oid2 = globally_unique_id();
|
ObjectID oid2 = globally_unique_id();
|
||||||
object_buffer obj_buffer;
|
ObjectBuffer obj_buffer;
|
||||||
|
|
||||||
object_id oid_array1[1] = {oid1};
|
ObjectID oid_array1[1] = {oid1};
|
||||||
object_id oid_array2[1] = {oid2};
|
ObjectID oid_array2[1] = {oid2};
|
||||||
|
|
||||||
int64_t data_size = 4;
|
int64_t data_size = 4;
|
||||||
uint8_t metadata[] = {5};
|
uint8_t metadata[] = {5};
|
||||||
|
@ -252,14 +252,14 @@ TEST plasma_get_tests(void) {
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST plasma_get_multiple_tests(void) {
|
TEST plasma_get_multiple_tests(void) {
|
||||||
plasma_connection *plasma_conn1 = plasma_connect(
|
PlasmaConnection *plasma_conn1 = plasma_connect(
|
||||||
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store1", "/tmp/manager1", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
plasma_connection *plasma_conn2 = plasma_connect(
|
PlasmaConnection *plasma_conn2 = plasma_connect(
|
||||||
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
"/tmp/store2", "/tmp/manager2", PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
object_id oid1 = globally_unique_id();
|
ObjectID oid1 = globally_unique_id();
|
||||||
object_id oid2 = globally_unique_id();
|
ObjectID oid2 = globally_unique_id();
|
||||||
object_id obj_ids[NUM_OBJ_REQUEST];
|
ObjectID obj_ids[NUM_OBJ_REQUEST];
|
||||||
object_buffer obj_buffer[NUM_OBJ_REQUEST];
|
ObjectBuffer obj_buffer[NUM_OBJ_REQUEST];
|
||||||
int obj1_first = 1, obj2_first = 2;
|
int obj1_first = 1, obj2_first = 2;
|
||||||
|
|
||||||
obj_ids[0] = oid1;
|
obj_ids[0] = oid1;
|
||||||
|
|
|
@ -23,7 +23,7 @@ SUITE(plasma_manager_tests);
|
||||||
const char *plasma_store_socket_name = "/tmp/plasma_store_socket_1";
|
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 *plasma_manager_socket_name_format = "/tmp/plasma_manager_socket_%d";
|
||||||
const char *manager_addr = "127.0.0.1";
|
const char *manager_addr = "127.0.0.1";
|
||||||
object_id oid;
|
ObjectID oid;
|
||||||
|
|
||||||
void wait_for_pollin(int fd) {
|
void wait_for_pollin(int fd) {
|
||||||
struct pollfd poll_list[1];
|
struct pollfd poll_list[1];
|
||||||
|
@ -49,12 +49,12 @@ typedef struct {
|
||||||
plasma_manager_state *state;
|
plasma_manager_state *state;
|
||||||
event_loop *loop;
|
event_loop *loop;
|
||||||
/* Accept a connection from the local manager on the remote manager. */
|
/* Accept a connection from the local manager on the remote manager. */
|
||||||
client_connection *write_conn;
|
ClientConnection *write_conn;
|
||||||
client_connection *read_conn;
|
ClientConnection *read_conn;
|
||||||
/* Connect a new client to the local plasma manager and mock a request to an
|
/* Connect a new client to the local plasma manager and mock a request to an
|
||||||
* object. */
|
* object. */
|
||||||
plasma_connection *plasma_conn;
|
PlasmaConnection *plasma_conn;
|
||||||
client_connection *client_conn;
|
ClientConnection *client_conn;
|
||||||
} plasma_mock;
|
} plasma_mock;
|
||||||
|
|
||||||
plasma_mock *init_plasma_mock(plasma_mock *remote_mock) {
|
plasma_mock *init_plasma_mock(plasma_mock *remote_mock) {
|
||||||
|
@ -77,7 +77,7 @@ plasma_mock *init_plasma_mock(plasma_mock *remote_mock) {
|
||||||
get_manager_connection(remote_mock->state, manager_addr, mock->port);
|
get_manager_connection(remote_mock->state, manager_addr, mock->port);
|
||||||
wait_for_pollin(mock->manager_remote_fd);
|
wait_for_pollin(mock->manager_remote_fd);
|
||||||
mock->read_conn =
|
mock->read_conn =
|
||||||
new_client_connection(mock->loop, mock->manager_remote_fd, mock->state,
|
ClientConnection_init(mock->loop, mock->manager_remote_fd, mock->state,
|
||||||
PLASMA_DEFAULT_RELEASE_DELAY);
|
PLASMA_DEFAULT_RELEASE_DELAY);
|
||||||
} else {
|
} else {
|
||||||
mock->write_conn = NULL;
|
mock->write_conn = NULL;
|
||||||
|
@ -89,7 +89,7 @@ plasma_mock *init_plasma_mock(plasma_mock *remote_mock) {
|
||||||
utstring_body(manager_socket_name), 0);
|
utstring_body(manager_socket_name), 0);
|
||||||
wait_for_pollin(mock->manager_local_fd);
|
wait_for_pollin(mock->manager_local_fd);
|
||||||
mock->client_conn =
|
mock->client_conn =
|
||||||
new_client_connection(mock->loop, mock->manager_local_fd, mock->state, 0);
|
ClientConnection_init(mock->loop, mock->manager_local_fd, mock->state, 0);
|
||||||
utstring_free(manager_socket_name);
|
utstring_free(manager_socket_name);
|
||||||
return mock;
|
return mock;
|
||||||
}
|
}
|
||||||
|
@ -134,11 +134,11 @@ TEST request_transfer_test(void) {
|
||||||
int read_fd = get_client_sock(remote_mock->read_conn);
|
int read_fd = get_client_sock(remote_mock->read_conn);
|
||||||
uint8_t *request_data =
|
uint8_t *request_data =
|
||||||
plasma_receive(read_fd, MessageType_PlasmaDataRequest);
|
plasma_receive(read_fd, MessageType_PlasmaDataRequest);
|
||||||
object_id oid2;
|
ObjectID oid2;
|
||||||
char *address;
|
char *address;
|
||||||
int port;
|
int port;
|
||||||
plasma_read_DataRequest(request_data, &oid2, &address, &port);
|
plasma_read_DataRequest(request_data, &oid2, &address, &port);
|
||||||
ASSERT(object_ids_equal(oid, oid2));
|
ASSERT(ObjectID_equal(oid, oid2));
|
||||||
free(address);
|
free(address);
|
||||||
/* Clean up. */
|
/* Clean up. */
|
||||||
utstring_free(addr);
|
utstring_free(addr);
|
||||||
|
@ -187,12 +187,12 @@ TEST request_transfer_retry_test(void) {
|
||||||
int read_fd = get_client_sock(remote_mock2->read_conn);
|
int read_fd = get_client_sock(remote_mock2->read_conn);
|
||||||
uint8_t *request_data =
|
uint8_t *request_data =
|
||||||
plasma_receive(read_fd, MessageType_PlasmaDataRequest);
|
plasma_receive(read_fd, MessageType_PlasmaDataRequest);
|
||||||
object_id oid2;
|
ObjectID oid2;
|
||||||
char *address;
|
char *address;
|
||||||
int port;
|
int port;
|
||||||
plasma_read_DataRequest(request_data, &oid2, &address, &port);
|
plasma_read_DataRequest(request_data, &oid2, &address, &port);
|
||||||
free(address);
|
free(address);
|
||||||
ASSERT(object_ids_equal(oid, oid2));
|
ASSERT(ObjectID_equal(oid, oid2));
|
||||||
/* Clean up. */
|
/* Clean up. */
|
||||||
utstring_free(addr0);
|
utstring_free(addr0);
|
||||||
utstring_free(addr1);
|
utstring_free(addr1);
|
||||||
|
@ -259,14 +259,14 @@ TEST object_notifications_test(void) {
|
||||||
int flags = fcntl(fd[1], F_GETFL, 0);
|
int flags = fcntl(fd[1], F_GETFL, 0);
|
||||||
CHECK(fcntl(fd[1], F_SETFL, flags | O_NONBLOCK) == 0);
|
CHECK(fcntl(fd[1], F_SETFL, flags | O_NONBLOCK) == 0);
|
||||||
|
|
||||||
object_id oid = globally_unique_id();
|
ObjectID oid = globally_unique_id();
|
||||||
object_info info = {.obj_id = oid,
|
ObjectInfo info = {.obj_id = oid,
|
||||||
.data_size = 10,
|
.data_size = 10,
|
||||||
.metadata_size = 1,
|
.metadata_size = 1,
|
||||||
.create_time = 0,
|
.create_time = 0,
|
||||||
.construct_duration = 0,
|
.construct_duration = 0,
|
||||||
.digest = {0},
|
.digest = {0},
|
||||||
.is_deletion = false};
|
.is_deletion = false};
|
||||||
|
|
||||||
/* Check that the object is not local at first. */
|
/* Check that the object is not local at first. */
|
||||||
bool is_local = is_object_local(local_mock->state, oid);
|
bool is_local = is_object_local(local_mock->state, oid);
|
||||||
|
|
|
@ -45,9 +45,9 @@ uint8_t *read_message_from_file(int fd, int message_type) {
|
||||||
return data;
|
return data;
|
||||||
}
|
}
|
||||||
|
|
||||||
plasma_object random_plasma_object(void) {
|
PlasmaObject random_plasma_object(void) {
|
||||||
int random = rand();
|
int random = rand();
|
||||||
plasma_object object;
|
PlasmaObject object;
|
||||||
memset(&object, 0, sizeof(object));
|
memset(&object, 0, sizeof(object));
|
||||||
object.handle.store_fd = random + 7;
|
object.handle.store_fd = random + 7;
|
||||||
object.handle.mmap_size = random + 42;
|
object.handle.mmap_size = random + 42;
|
||||||
|
@ -60,18 +60,18 @@ plasma_object random_plasma_object(void) {
|
||||||
|
|
||||||
TEST plasma_create_request_test(void) {
|
TEST plasma_create_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
int64_t data_size1 = 42;
|
int64_t data_size1 = 42;
|
||||||
int64_t metadata_size1 = 11;
|
int64_t metadata_size1 = 11;
|
||||||
plasma_send_CreateRequest(fd, g_B, object_id1, data_size1, metadata_size1);
|
plasma_send_CreateRequest(fd, g_B, object_id1, data_size1, metadata_size1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaCreateRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaCreateRequest);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
int64_t data_size2;
|
int64_t data_size2;
|
||||||
int64_t metadata_size2;
|
int64_t metadata_size2;
|
||||||
plasma_read_CreateRequest(data, &object_id2, &data_size2, &metadata_size2);
|
plasma_read_CreateRequest(data, &object_id2, &data_size2, &metadata_size2);
|
||||||
ASSERT_EQ(data_size1, data_size2);
|
ASSERT_EQ(data_size1, data_size2);
|
||||||
ASSERT_EQ(metadata_size1, metadata_size2);
|
ASSERT_EQ(metadata_size1, metadata_size2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -79,16 +79,16 @@ TEST plasma_create_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_create_reply_test(void) {
|
TEST plasma_create_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
plasma_object object1 = random_plasma_object();
|
PlasmaObject object1 = random_plasma_object();
|
||||||
plasma_send_CreateReply(fd, g_B, object_id1, &object1, 0);
|
plasma_send_CreateReply(fd, g_B, object_id1, &object1, 0);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaCreateReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaCreateReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
plasma_object object2;
|
PlasmaObject object2;
|
||||||
memset(&object2, 0, sizeof(object2));
|
memset(&object2, 0, sizeof(object2));
|
||||||
int error_code;
|
int error_code;
|
||||||
plasma_read_CreateReply(data, &object_id2, &object2, &error_code);
|
plasma_read_CreateReply(data, &object_id2, &object2, &error_code);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(memcmp(&object1, &object2, sizeof(object1)) == 0);
|
ASSERT(memcmp(&object1, &object2, sizeof(object1)) == 0);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -97,15 +97,15 @@ TEST plasma_create_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_seal_request_test(void) {
|
TEST plasma_seal_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
unsigned char digest1[DIGEST_SIZE];
|
unsigned char digest1[DIGEST_SIZE];
|
||||||
memset(&digest1[0], 7, DIGEST_SIZE);
|
memset(&digest1[0], 7, DIGEST_SIZE);
|
||||||
plasma_send_SealRequest(fd, g_B, object_id1, &digest1[0]);
|
plasma_send_SealRequest(fd, g_B, object_id1, &digest1[0]);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaSealRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaSealRequest);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
unsigned char digest2[DIGEST_SIZE];
|
unsigned char digest2[DIGEST_SIZE];
|
||||||
plasma_read_SealRequest(data, &object_id2, &digest2[0]);
|
plasma_read_SealRequest(data, &object_id2, &digest2[0]);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(memcmp(&digest1[0], &digest2[0], DIGEST_SIZE) == 0);
|
ASSERT(memcmp(&digest1[0], &digest2[0], DIGEST_SIZE) == 0);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -114,14 +114,14 @@ TEST plasma_seal_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_seal_reply_test(void) {
|
TEST plasma_seal_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
int error1 = 5;
|
int error1 = 5;
|
||||||
plasma_send_SealReply(fd, g_B, object_id1, error1);
|
plasma_send_SealReply(fd, g_B, object_id1, error1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaSealReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaSealReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
int error2;
|
int error2;
|
||||||
plasma_read_SealReply(data, &object_id2, &error2);
|
plasma_read_SealReply(data, &object_id2, &error2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(error1 == error2);
|
ASSERT(error1 == error2);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -130,18 +130,18 @@ TEST plasma_seal_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_get_request_test(void) {
|
TEST plasma_get_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_ids[2];
|
ObjectID object_ids[2];
|
||||||
object_ids[0] = globally_unique_id();
|
object_ids[0] = globally_unique_id();
|
||||||
object_ids[1] = globally_unique_id();
|
object_ids[1] = globally_unique_id();
|
||||||
int64_t timeout_ms = 1234;
|
int64_t timeout_ms = 1234;
|
||||||
plasma_send_GetRequest(fd, g_B, object_ids, 2, timeout_ms);
|
plasma_send_GetRequest(fd, g_B, object_ids, 2, timeout_ms);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaGetRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaGetRequest);
|
||||||
int64_t num_objects;
|
int64_t num_objects;
|
||||||
object_id object_ids_return[2];
|
ObjectID object_ids_return[2];
|
||||||
int64_t timeout_ms_return;
|
int64_t timeout_ms_return;
|
||||||
plasma_read_GetRequest(data, &object_ids_return[0], &timeout_ms_return, 2);
|
plasma_read_GetRequest(data, &object_ids_return[0], &timeout_ms_return, 2);
|
||||||
ASSERT(object_ids_equal(object_ids[0], object_ids_return[0]));
|
ASSERT(ObjectID_equal(object_ids[0], object_ids_return[0]));
|
||||||
ASSERT(object_ids_equal(object_ids[1], object_ids_return[1]));
|
ASSERT(ObjectID_equal(object_ids[1], object_ids_return[1]));
|
||||||
ASSERT(timeout_ms == timeout_ms_return);
|
ASSERT(timeout_ms == timeout_ms_return);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -150,25 +150,25 @@ TEST plasma_get_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_get_reply_test(void) {
|
TEST plasma_get_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_ids[2];
|
ObjectID object_ids[2];
|
||||||
object_ids[0] = globally_unique_id();
|
object_ids[0] = globally_unique_id();
|
||||||
object_ids[1] = globally_unique_id();
|
object_ids[1] = globally_unique_id();
|
||||||
plasma_object plasma_objects[2];
|
PlasmaObject plasma_objects[2];
|
||||||
plasma_objects[0] = random_plasma_object();
|
plasma_objects[0] = random_plasma_object();
|
||||||
plasma_objects[1] = random_plasma_object();
|
plasma_objects[1] = random_plasma_object();
|
||||||
plasma_send_GetReply(fd, g_B, object_ids, plasma_objects, 2);
|
plasma_send_GetReply(fd, g_B, object_ids, plasma_objects, 2);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaGetReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaGetReply);
|
||||||
int64_t num_objects = plasma_read_GetRequest_num_objects(data);
|
int64_t num_objects = plasma_read_GetRequest_num_objects(data);
|
||||||
object_id object_ids_return[num_objects];
|
ObjectID object_ids_return[num_objects];
|
||||||
plasma_object plasma_objects_return[2];
|
PlasmaObject plasma_objects_return[2];
|
||||||
plasma_read_GetReply(data, object_ids_return, &plasma_objects_return[0],
|
plasma_read_GetReply(data, object_ids_return, &plasma_objects_return[0],
|
||||||
num_objects);
|
num_objects);
|
||||||
ASSERT(object_ids_equal(object_ids[0], object_ids_return[0]));
|
ASSERT(ObjectID_equal(object_ids[0], object_ids_return[0]));
|
||||||
ASSERT(object_ids_equal(object_ids[1], object_ids_return[1]));
|
ASSERT(ObjectID_equal(object_ids[1], object_ids_return[1]));
|
||||||
ASSERT(memcmp(&plasma_objects[0], &plasma_objects_return[0],
|
ASSERT(memcmp(&plasma_objects[0], &plasma_objects_return[0],
|
||||||
sizeof(plasma_object)) == 0);
|
sizeof(PlasmaObject)) == 0);
|
||||||
ASSERT(memcmp(&plasma_objects[1], &plasma_objects_return[1],
|
ASSERT(memcmp(&plasma_objects[1], &plasma_objects_return[1],
|
||||||
sizeof(plasma_object)) == 0);
|
sizeof(PlasmaObject)) == 0);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -176,12 +176,12 @@ TEST plasma_get_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_release_request_test(void) {
|
TEST plasma_release_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
plasma_send_ReleaseRequest(fd, g_B, object_id1);
|
plasma_send_ReleaseRequest(fd, g_B, object_id1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaReleaseRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaReleaseRequest);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
plasma_read_ReleaseRequest(data, &object_id2);
|
plasma_read_ReleaseRequest(data, &object_id2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -189,14 +189,14 @@ TEST plasma_release_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_release_reply_test(void) {
|
TEST plasma_release_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
int error1 = 5;
|
int error1 = 5;
|
||||||
plasma_send_ReleaseReply(fd, g_B, object_id1, error1);
|
plasma_send_ReleaseReply(fd, g_B, object_id1, error1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaReleaseReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaReleaseReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
int error2;
|
int error2;
|
||||||
plasma_read_ReleaseReply(data, &object_id2, &error2);
|
plasma_read_ReleaseReply(data, &object_id2, &error2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(error1 == error2);
|
ASSERT(error1 == error2);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -205,12 +205,12 @@ TEST plasma_release_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_delete_request_test(void) {
|
TEST plasma_delete_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
plasma_send_DeleteRequest(fd, g_B, object_id1);
|
plasma_send_DeleteRequest(fd, g_B, object_id1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDeleteRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDeleteRequest);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
plasma_read_DeleteRequest(data, &object_id2);
|
plasma_read_DeleteRequest(data, &object_id2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -218,14 +218,14 @@ TEST plasma_delete_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_delete_reply_test(void) {
|
TEST plasma_delete_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
int error1 = 5;
|
int error1 = 5;
|
||||||
plasma_send_DeleteReply(fd, g_B, object_id1, error1);
|
plasma_send_DeleteReply(fd, g_B, object_id1, error1);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDeleteReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDeleteReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
int error2;
|
int error2;
|
||||||
plasma_read_DeleteReply(data, &object_id2, &error2);
|
plasma_read_DeleteReply(data, &object_id2, &error2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(error1 == error2);
|
ASSERT(error1 == error2);
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
|
@ -234,16 +234,16 @@ TEST plasma_delete_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_status_request_test(void) {
|
TEST plasma_status_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_ids[2];
|
ObjectID object_ids[2];
|
||||||
object_ids[0] = globally_unique_id();
|
object_ids[0] = globally_unique_id();
|
||||||
object_ids[1] = globally_unique_id();
|
object_ids[1] = globally_unique_id();
|
||||||
plasma_send_StatusRequest(fd, g_B, object_ids, 2);
|
plasma_send_StatusRequest(fd, g_B, object_ids, 2);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaStatusRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaStatusRequest);
|
||||||
int64_t num_objects = plasma_read_StatusRequest_num_objects(data);
|
int64_t num_objects = plasma_read_StatusRequest_num_objects(data);
|
||||||
object_id object_ids_read[num_objects];
|
ObjectID object_ids_read[num_objects];
|
||||||
plasma_read_StatusRequest(data, object_ids_read, num_objects);
|
plasma_read_StatusRequest(data, object_ids_read, num_objects);
|
||||||
ASSERT(object_ids_equal(object_ids[0], object_ids_read[0]));
|
ASSERT(ObjectID_equal(object_ids[0], object_ids_read[0]));
|
||||||
ASSERT(object_ids_equal(object_ids[1], object_ids_read[1]));
|
ASSERT(ObjectID_equal(object_ids[1], object_ids_read[1]));
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -251,19 +251,19 @@ TEST plasma_status_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_status_reply_test(void) {
|
TEST plasma_status_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_ids[2];
|
ObjectID object_ids[2];
|
||||||
object_ids[0] = globally_unique_id();
|
object_ids[0] = globally_unique_id();
|
||||||
object_ids[1] = globally_unique_id();
|
object_ids[1] = globally_unique_id();
|
||||||
int object_statuses[2] = {42, 43};
|
int object_statuses[2] = {42, 43};
|
||||||
plasma_send_StatusReply(fd, g_B, object_ids, object_statuses, 2);
|
plasma_send_StatusReply(fd, g_B, object_ids, object_statuses, 2);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaStatusReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaStatusReply);
|
||||||
int64_t num_objects = plasma_read_StatusReply_num_objects(data);
|
int64_t num_objects = plasma_read_StatusReply_num_objects(data);
|
||||||
object_id object_ids_read[num_objects];
|
ObjectID object_ids_read[num_objects];
|
||||||
int object_statuses_read[num_objects];
|
int object_statuses_read[num_objects];
|
||||||
plasma_read_StatusReply(data, object_ids_read, object_statuses_read,
|
plasma_read_StatusReply(data, object_ids_read, object_statuses_read,
|
||||||
num_objects);
|
num_objects);
|
||||||
ASSERT(object_ids_equal(object_ids[0], object_ids_read[0]));
|
ASSERT(ObjectID_equal(object_ids[0], object_ids_read[0]));
|
||||||
ASSERT(object_ids_equal(object_ids[1], object_ids_read[1]));
|
ASSERT(ObjectID_equal(object_ids[1], object_ids_read[1]));
|
||||||
ASSERT_EQ(object_statuses[0], object_statuses_read[0]);
|
ASSERT_EQ(object_statuses[0], object_statuses_read[0]);
|
||||||
ASSERT_EQ(object_statuses[1], object_statuses_read[1]);
|
ASSERT_EQ(object_statuses[1], object_statuses_read[1]);
|
||||||
free(data);
|
free(data);
|
||||||
|
@ -274,7 +274,7 @@ TEST plasma_status_reply_test(void) {
|
||||||
TEST plasma_evict_request_test(void) {
|
TEST plasma_evict_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
int64_t num_bytes = 111;
|
int64_t num_bytes = 111;
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
plasma_send_EvictRequest(fd, g_B, num_bytes);
|
plasma_send_EvictRequest(fd, g_B, num_bytes);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaEvictRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaEvictRequest);
|
||||||
int64_t num_bytes_received;
|
int64_t num_bytes_received;
|
||||||
|
@ -288,7 +288,7 @@ TEST plasma_evict_request_test(void) {
|
||||||
TEST plasma_evict_reply_test(void) {
|
TEST plasma_evict_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
int64_t num_bytes = 111;
|
int64_t num_bytes = 111;
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
plasma_send_EvictReply(fd, g_B, num_bytes);
|
plasma_send_EvictReply(fd, g_B, num_bytes);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaEvictReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaEvictReply);
|
||||||
int64_t num_bytes_received;
|
int64_t num_bytes_received;
|
||||||
|
@ -301,15 +301,15 @@ TEST plasma_evict_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_fetch_request_test(void) {
|
TEST plasma_fetch_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_ids[2];
|
ObjectID object_ids[2];
|
||||||
object_ids[0] = globally_unique_id();
|
object_ids[0] = globally_unique_id();
|
||||||
object_ids[1] = globally_unique_id();
|
object_ids[1] = globally_unique_id();
|
||||||
plasma_send_FetchRequest(fd, g_B, object_ids, 2);
|
plasma_send_FetchRequest(fd, g_B, object_ids, 2);
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaFetchRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaFetchRequest);
|
||||||
object_id object_ids_read[2];
|
ObjectID object_ids_read[2];
|
||||||
plasma_read_FetchRequest(data, &object_ids_read[0], 2);
|
plasma_read_FetchRequest(data, &object_ids_read[0], 2);
|
||||||
ASSERT(object_ids_equal(object_ids[0], object_ids_read[0]));
|
ASSERT(ObjectID_equal(object_ids[0], object_ids_read[0]));
|
||||||
ASSERT(object_ids_equal(object_ids[1], object_ids_read[1]));
|
ASSERT(ObjectID_equal(object_ids[1], object_ids_read[1]));
|
||||||
free(data);
|
free(data);
|
||||||
close(fd);
|
close(fd);
|
||||||
PASS();
|
PASS();
|
||||||
|
@ -317,7 +317,7 @@ TEST plasma_fetch_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_wait_request_test(void) {
|
TEST plasma_wait_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_request object_requests[2];
|
ObjectRequest object_requests[2];
|
||||||
object_requests[0].object_id = globally_unique_id();
|
object_requests[0].object_id = globally_unique_id();
|
||||||
object_requests[0].type = PLASMA_QUERY_ANYWHERE;
|
object_requests[0].type = PLASMA_QUERY_ANYWHERE;
|
||||||
object_requests[1].object_id = globally_unique_id();
|
object_requests[1].object_id = globally_unique_id();
|
||||||
|
@ -328,17 +328,17 @@ TEST plasma_wait_request_test(void) {
|
||||||
timeout_ms);
|
timeout_ms);
|
||||||
/* Read message back. */
|
/* Read message back. */
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaWaitRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaWaitRequest);
|
||||||
object_request object_requests_read[2];
|
ObjectRequest object_requests_read[2];
|
||||||
int num_object_ids_read = plasma_read_WaitRequest_num_object_ids(data);
|
int num_object_ids_read = plasma_read_WaitRequest_num_object_ids(data);
|
||||||
ASSERT_EQ(num_object_ids_read, 2);
|
ASSERT_EQ(num_object_ids_read, 2);
|
||||||
int num_ready_objects_read;
|
int num_ready_objects_read;
|
||||||
int64_t timeout_ms_read;
|
int64_t timeout_ms_read;
|
||||||
plasma_read_WaitRequest(data, &object_requests_read[0], num_object_ids_read,
|
plasma_read_WaitRequest(data, &object_requests_read[0], num_object_ids_read,
|
||||||
&timeout_ms_read, &num_ready_objects_read);
|
&timeout_ms_read, &num_ready_objects_read);
|
||||||
ASSERT(object_ids_equal(object_requests[0].object_id,
|
ASSERT(ObjectID_equal(object_requests[0].object_id,
|
||||||
object_requests_read[0].object_id));
|
object_requests_read[0].object_id));
|
||||||
ASSERT(object_ids_equal(object_requests[1].object_id,
|
ASSERT(ObjectID_equal(object_requests[1].object_id,
|
||||||
object_requests_read[1].object_id));
|
object_requests_read[1].object_id));
|
||||||
ASSERT(object_requests[0].type == object_requests_read[0].type);
|
ASSERT(object_requests[0].type == object_requests_read[0].type);
|
||||||
ASSERT(object_requests[1].type == object_requests_read[1].type);
|
ASSERT(object_requests[1].type == object_requests_read[1].type);
|
||||||
free(data);
|
free(data);
|
||||||
|
@ -348,7 +348,7 @@ TEST plasma_wait_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_wait_reply_test(void) {
|
TEST plasma_wait_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_request object_replies1[2];
|
ObjectRequest object_replies1[2];
|
||||||
object_replies1[0].object_id = globally_unique_id();
|
object_replies1[0].object_id = globally_unique_id();
|
||||||
object_replies1[0].status = ObjectStatus_Local;
|
object_replies1[0].status = ObjectStatus_Local;
|
||||||
object_replies1[1].object_id = globally_unique_id();
|
object_replies1[1].object_id = globally_unique_id();
|
||||||
|
@ -357,13 +357,13 @@ TEST plasma_wait_reply_test(void) {
|
||||||
plasma_send_WaitReply(fd, g_B, object_replies1, num_ready_objects1);
|
plasma_send_WaitReply(fd, g_B, object_replies1, num_ready_objects1);
|
||||||
/* Read message back. */
|
/* Read message back. */
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaWaitReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaWaitReply);
|
||||||
object_request object_replies2[2];
|
ObjectRequest object_replies2[2];
|
||||||
int num_ready_objects_read2;
|
int num_ready_objects_read2;
|
||||||
plasma_read_WaitReply(data, &object_replies2[0], &num_ready_objects_read2);
|
plasma_read_WaitReply(data, &object_replies2[0], &num_ready_objects_read2);
|
||||||
ASSERT(object_ids_equal(object_replies1[0].object_id,
|
ASSERT(ObjectID_equal(object_replies1[0].object_id,
|
||||||
object_replies2[0].object_id));
|
object_replies2[0].object_id));
|
||||||
ASSERT(object_ids_equal(object_replies1[1].object_id,
|
ASSERT(ObjectID_equal(object_replies1[1].object_id,
|
||||||
object_replies2[1].object_id));
|
object_replies2[1].object_id));
|
||||||
ASSERT(object_replies1[0].status == object_replies2[0].status);
|
ASSERT(object_replies1[0].status == object_replies2[0].status);
|
||||||
ASSERT(object_replies1[1].status == object_replies2[1].status);
|
ASSERT(object_replies1[1].status == object_replies2[1].status);
|
||||||
free(data);
|
free(data);
|
||||||
|
@ -373,17 +373,17 @@ TEST plasma_wait_reply_test(void) {
|
||||||
|
|
||||||
TEST plasma_data_request_test(void) {
|
TEST plasma_data_request_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
const char *address1 = "address1";
|
const char *address1 = "address1";
|
||||||
int port1 = 12345;
|
int port1 = 12345;
|
||||||
plasma_send_DataRequest(fd, g_B, object_id1, address1, port1);
|
plasma_send_DataRequest(fd, g_B, object_id1, address1, port1);
|
||||||
/* Reading message back. */
|
/* Reading message back. */
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDataRequest);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDataRequest);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
char *address2;
|
char *address2;
|
||||||
int port2;
|
int port2;
|
||||||
plasma_read_DataRequest(data, &object_id2, &address2, &port2);
|
plasma_read_DataRequest(data, &object_id2, &address2, &port2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(strcmp(address1, address2) == 0);
|
ASSERT(strcmp(address1, address2) == 0);
|
||||||
ASSERT(port1 == port2);
|
ASSERT(port1 == port2);
|
||||||
free(address2);
|
free(address2);
|
||||||
|
@ -394,17 +394,17 @@ TEST plasma_data_request_test(void) {
|
||||||
|
|
||||||
TEST plasma_data_reply_test(void) {
|
TEST plasma_data_reply_test(void) {
|
||||||
int fd = create_temp_file();
|
int fd = create_temp_file();
|
||||||
object_id object_id1 = globally_unique_id();
|
ObjectID object_id1 = globally_unique_id();
|
||||||
int64_t object_size1 = 146;
|
int64_t object_size1 = 146;
|
||||||
int64_t metadata_size1 = 198;
|
int64_t metadata_size1 = 198;
|
||||||
plasma_send_DataReply(fd, g_B, object_id1, object_size1, metadata_size1);
|
plasma_send_DataReply(fd, g_B, object_id1, object_size1, metadata_size1);
|
||||||
/* Reading message back. */
|
/* Reading message back. */
|
||||||
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDataReply);
|
uint8_t *data = read_message_from_file(fd, MessageType_PlasmaDataReply);
|
||||||
object_id object_id2;
|
ObjectID object_id2;
|
||||||
int64_t object_size2;
|
int64_t object_size2;
|
||||||
int64_t metadata_size2;
|
int64_t metadata_size2;
|
||||||
plasma_read_DataReply(data, &object_id2, &object_size2, &metadata_size2);
|
plasma_read_DataReply(data, &object_id2, &object_size2, &metadata_size2);
|
||||||
ASSERT(object_ids_equal(object_id1, object_id2));
|
ASSERT(ObjectID_equal(object_id1, object_id2));
|
||||||
ASSERT(object_size1 == object_size2);
|
ASSERT(object_size1 == object_size2);
|
||||||
ASSERT(metadata_size1 == metadata_size2);
|
ASSERT(metadata_size1 == metadata_size2);
|
||||||
free(data);
|
free(data);
|
||||||
|
|
Loading…
Add table
Reference in a new issue