Skip to content

Commit

Permalink
Implement object table notification subscriptions and switch to using…
Browse files Browse the repository at this point in the history
… Redis modules for object table. (ray-project#134)

* Implement RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS.

* Call object_table_request_notifications from plasma manager.

* Use Redis modules for object table.

* Cleaning up code.

* More checks.

* Formatting.

* Make object table tests pass.

* Formatting.

* Add prefix to the object notification channel name.

* Formatting.

* Fixes.

* Increase time in redismodule test.
  • Loading branch information
robertnishihara authored and pcmoritz committed Dec 19, 2016
1 parent c89bf4e commit 269f37e
Show file tree
Hide file tree
Showing 10 changed files with 665 additions and 461 deletions.
1 change: 1 addition & 0 deletions .clang-format
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
BasedOnStyle: Chromium
ColumnLimit: 80
DerivePointerAlignment: false
IndentCaseLabels: false
PointerAlignment: Right
Expand Down
194 changes: 155 additions & 39 deletions src/common/redis_module/ray_redis_module.c
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
#include "redismodule.h"

#include <stdbool.h>
#include <string.h>

/**
Expand All @@ -22,9 +23,11 @@
#define DB_CLIENT_PREFIX "CL:"
#define OBJECT_INFO_PREFIX "OI:"
#define OBJECT_LOCATION_PREFIX "OL:"
#define OBJECT_SUBSCRIBE_PREFIX "OS:"
#define OBJECT_NOTIFICATION_PREFIX "ON:"
#define TASK_PREFIX "TT:"

#define OBJECT_CHANNEL_PREFIX "OC:"

#define CHECK_ERROR(STATUS, MESSAGE) \
if ((STATUS) == REDISMODULE_ERR) { \
return RedisModule_ReplyWithError(ctx, (MESSAGE)); \
Expand Down Expand Up @@ -211,6 +214,56 @@ int ObjectTableLookup_RedisCommand(RedisModuleCtx *ctx,
return REDISMODULE_OK;
}

/**
* Publish a notification to a client's object notification channel if at least
* one manager is listed as having the object in the object table.
*
* @param ctx The Redis context.
* @param client_id The ID of the client that is being notified.
* @param object_id The object ID of interest.
* @param key The opened key for the entry in the object table corresponding to
* the object ID of interest.
* @return True if the publish was successful and false otherwise.
*/
bool PublishObjectNotification(RedisModuleCtx *ctx,
RedisModuleString *client_id,
RedisModuleString *object_id,
RedisModuleKey *key) {
/* Create a string formatted as "<object id> MANAGERS <manager id1>
* <manager id2> ..." */
RedisModuleString *manager_list =
RedisModule_CreateStringFromString(ctx, object_id);
RedisModule_StringAppendBuffer(ctx, manager_list, " MANAGERS",
strlen(" MANAGERS"));

CHECK_ERROR(
RedisModule_ZsetFirstInScoreRange(key, REDISMODULE_NEGATIVE_INFINITE,
REDISMODULE_POSITIVE_INFINITE, 1, 1),
"Unable to initialize zset iterator");

/* Loop over the managers in the object table for this object ID. */
do {
RedisModuleString *curr = RedisModule_ZsetRangeCurrentElement(key, NULL);
RedisModule_StringAppendBuffer(ctx, manager_list, " ", 1);
size_t size;
const char *val = RedisModule_StringPtrLen(curr, &size);
RedisModule_StringAppendBuffer(ctx, manager_list, val, size);
} while (RedisModule_ZsetRangeNext(key));

/* Publish the notification to the clients notification channel.
* TODO(rkn): These notifications could be batched together. */
RedisModuleString *channel_name =
CreatePrefixedString(ctx, OBJECT_CHANNEL_PREFIX, client_id);
RedisModuleCallReply *reply;
reply = RedisModule_Call(ctx, "PUBLISH", "ss", channel_name, manager_list);
RedisModule_FreeString(ctx, channel_name);
RedisModule_FreeString(ctx, manager_list);
if (reply == NULL) {
return false;
}
return true;
}

/**
* Add a new entry to the object table or update an existing one.
*
Expand Down Expand Up @@ -275,47 +328,110 @@ int ObjectTableAdd_RedisCommand(RedisModuleCtx *ctx,
/* Sets are not implemented yet, so we use ZSETs instead. */
RedisModule_ZsetAdd(table_key, 0.0, manager, NULL);

/* Build the PUBLISH topic and message for object table subscribers. The
* topic is a string in the format "OBJECT_LOCATION_PREFIX:<object ID>". The
* message is a string in the format: "<manager ID> <manager ID> ... <manager
* ID>". */
RedisModuleString *publish_topic =
CreatePrefixedString(ctx, OBJECT_LOCATION_PREFIX, object_id);
const char *MANAGERS = "MANAGERS";
RedisModuleString *publish =
RedisModule_CreateString(ctx, MANAGERS, strlen(MANAGERS));
CHECK_ERROR(RedisModule_ZsetFirstInScoreRange(
table_key, REDISMODULE_NEGATIVE_INFINITE,
REDISMODULE_POSITIVE_INFINITE, 1, 1),
"Unable to initialize zset iterator");
do {
RedisModuleString *curr =
RedisModule_ZsetRangeCurrentElement(table_key, NULL);
RedisModule_StringAppendBuffer(ctx, publish, " ", 1);
size_t size;
const char *val = RedisModule_StringPtrLen(curr, &size);
RedisModule_StringAppendBuffer(ctx, publish, val, size);
} while (RedisModule_ZsetRangeNext(table_key));

RedisModuleCallReply *reply =
RedisModule_Call(ctx, "PUBLISH", "ss", publish_topic, publish);
RedisModule_FreeString(ctx, publish);
RedisModule_FreeString(ctx, publish_topic);
RedisModule_CloseKey(table_key);
if (reply == NULL) {
return RedisModule_ReplyWithError(ctx, "PUBLISH unsuccessful");
/* Get the zset of clients that requested a notification about the
* availability of this object. */
RedisModuleKey *object_notification_key =
OpenPrefixedKey(ctx, OBJECT_NOTIFICATION_PREFIX, object_id,
REDISMODULE_READ | REDISMODULE_WRITE);
/* If the zset exists, initialize the key to iterate over the zset. */
int object_notification_keytype =
RedisModule_KeyType(object_notification_key);
if (object_notification_keytype != REDISMODULE_KEYTYPE_EMPTY) {
CHECK_ERROR(RedisModule_ZsetFirstInScoreRange(
object_notification_key, REDISMODULE_NEGATIVE_INFINITE,
REDISMODULE_POSITIVE_INFINITE, 1, 1),
"Unable to initialize zset iterator");
/* Iterate over the list of clients that requested notifiations about the
* availability of this object, and publish notifications to their object
* notification channels. */
do {
RedisModuleString *client_id =
RedisModule_ZsetRangeCurrentElement(object_notification_key, NULL);
/* TODO(rkn): Some computation could be saved by batching the string
* constructions in the multiple calls to PublishObjectNotification
* together. */
bool success =
PublishObjectNotification(ctx, client_id, object_id, table_key);
if (!success) {
/* The publish failed somehow. */
RedisModule_CloseKey(object_notification_key);
return RedisModule_ReplyWithError(ctx, "PUBLISH unsuccessful");
}
} while (RedisModule_ZsetRangeNext(object_notification_key));
/* Now that the clients have been notified, remove the zset of clients
* waiting for notifications. */
CHECK_ERROR(RedisModule_DeleteKey(object_notification_key),
"Unable to delete zset key.");
RedisModule_CloseKey(object_notification_key);
}

RedisModule_ReplyWithSimpleString(ctx, "OK");
return REDISMODULE_OK;
}

int ObjectTableSubscribe_RedisCommand(RedisModuleCtx *ctx,
RedisModuleString **argv,
int argc) {
REDISMODULE_NOT_USED(ctx);
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
/**
* Request notifications about the presence of some object IDs. This command
* takes a list of object IDs. There will be an immediate reply acknowledging
* the call and containing a list of all the object IDs that are already
* present in the object table along with vectors of the plasma managers that
* contain each object. For each object ID that is not already present in the
* object table, there will be a separate subsequent reply that returns the list
* of manager vectors conaining the object ID, and this will be called as soon
* as the object is added to the object table.
*
* This is called from a client with the command:
*
* RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS <client id> <object id1>
* <object id2> ...
*
* @param client_id The ID of the client that is requesting the notifications.
* @param object_id(n) The ID of the nth object ID that is passed to this
* command. This command can take any number of object IDs.
* @return OK if the operation was successful.
*/
int ObjectTableRequestNotifications_RedisCommand(RedisModuleCtx *ctx,
RedisModuleString **argv,
int argc) {
if (argc < 3) {
return RedisModule_WrongArity(ctx);
}

/* The first argument is the client ID. The other arguments are object IDs. */
RedisModuleString *client_id = argv[1];

/* Loop over the object ID arguments to this command. */
for (int i = 2; i < argc; ++i) {
RedisModuleString *object_id = argv[i];
RedisModuleKey *key = OpenPrefixedKey(ctx, OBJECT_LOCATION_PREFIX,
object_id, REDISMODULE_READ);
int keytype = RedisModule_KeyType(key);
if (keytype == REDISMODULE_KEYTYPE_EMPTY ||
RedisModule_ValueLength(key) == 0) {
/* This object ID is currently not present, so make a note that this
* client should be notified when this object ID becomes available. */
RedisModuleKey *object_notification_key =
OpenPrefixedKey(ctx, OBJECT_NOTIFICATION_PREFIX, object_id,
REDISMODULE_READ | REDISMODULE_WRITE);
/* Add this client to the list of clients that will be notified when this
* object becomes available. */
CHECK_ERROR(
RedisModule_ZsetAdd(object_notification_key, 0.0, client_id, NULL),
"ZsetAdd failed.");
RedisModule_CloseKey(object_notification_key);
} else {
/* Publish a notification to the client's object notification channel. */
bool success = PublishObjectNotification(ctx, client_id, object_id, key);
if (!success) {
/* The publish failed somehow. */
RedisModule_CloseKey(key);
return RedisModule_ReplyWithError(ctx, "PUBLISH unsuccessful");
}
}
/* Clean up. */
RedisModule_CloseKey(key);
}

RedisModule_ReplyWithSimpleString(ctx, "OK");
return REDISMODULE_OK;
}

Expand Down Expand Up @@ -649,9 +765,9 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx,
return REDISMODULE_ERR;
}

if (RedisModule_CreateCommand(ctx, "ray.object_table_subscribe",
ObjectTableSubscribe_RedisCommand, "pubsub", 0,
0, 0) == REDISMODULE_ERR) {
if (RedisModule_CreateCommand(ctx, "ray.object_table_request_notifications",
ObjectTableRequestNotifications_RedisCommand,
"write pubsub", 0, 0, 0) == REDISMODULE_ERR) {
return REDISMODULE_ERR;
}

Expand Down
27 changes: 21 additions & 6 deletions src/common/redis_module/runtest.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
OBJECT_LOCATION_PREFIX = "OL:"
OBJECT_SUBSCRIBE_PREFIX = "OS:"
TASK_PREFIX = "TT:"
OBJECT_CHANNEL_PREFIX = "OC:"

class TestGlobalStateStore(unittest.TestCase):

Expand All @@ -30,7 +31,7 @@ def setUp(self):
self.redis_process = subprocess.Popen([redis_path,
"--port", str(redis_port),
"--loadmodule", module_path])
time.sleep(0.5)
time.sleep(1.5)
self.redis = redis.StrictRedis(host="localhost", port=redis_port, db=0)

def tearDown(self):
Expand Down Expand Up @@ -84,15 +85,29 @@ def testObjectTableAddAndLookup(self):
with self.assertRaises(redis.ResponseError):
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id3", 1, "\x00hash2", "manager_id1")

def testObjectTableSubscribe(self):
def testObjectTableSubscribeToNotifications(self):
p = self.redis.pubsub()
# Subscribe to an object ID.
p.psubscribe("{0}*".format(OBJECT_LOCATION_PREFIX))
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id1", 1, "hash1", "manager_id1")
p.psubscribe("{}manager_id1".format(OBJECT_CHANNEL_PREFIX))
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id1", 1, "hash1", "manager_id2")
# Receive the acknowledgement message.
self.assertEqual(p.get_message()["data"], 1)
# Receive the actual data.
self.assertEqual(p.get_message()["data"], b"MANAGERS manager_id1")
# Request a notification and receive the data.
self.redis.execute_command("RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS", "manager_id1", "object_id1")
self.assertEqual(p.get_message()["data"], b"object_id1 MANAGERS manager_id2")
# Request a notification for an object that isn't there. Then add the object
# and receive the data. Only the first call to RAY.OBJECT_TABLE_ADD should
# trigger notifications.
self.redis.execute_command("RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS", "manager_id1", "object_id2", "object_id3")
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id3", 1, "hash1", "manager_id1")
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id3", 1, "hash1", "manager_id2")
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id3", 1, "hash1", "manager_id3")
self.assertEqual(p.get_message()["data"], b"object_id3 MANAGERS manager_id1")
self.redis.execute_command("RAY.OBJECT_TABLE_ADD", "object_id2", 1, "hash1", "manager_id3")
self.assertEqual(p.get_message()["data"], b"object_id2 MANAGERS manager_id3")
# Request notifications for object_id3 again.
self.redis.execute_command("RAY.OBJECT_TABLE_REQUEST_NOTIFICATIONS", "manager_id1", "object_id3")
self.assertEqual(p.get_message()["data"], b"object_id3 MANAGERS manager_id1 manager_id2 manager_id3")

def testResultTableAddAndLookup(self):
response = self.redis.execute_command("RAY.RESULT_TABLE_LOOKUP", "object_id1")
Expand Down
31 changes: 23 additions & 8 deletions src/common/state/object_table.c
Original file line number Diff line number Diff line change
Expand Up @@ -14,23 +14,22 @@ void object_table_lookup(db_handle *db_handle,

void object_table_add(db_handle *db_handle,
object_id object_id,
int64_t data_size,
int64_t object_size,
unsigned char digest[],
retry_info *retry,
object_table_done_callback done_callback,
void *user_context) {
CHECK(db_handle != NULL);

object_info *info = malloc(sizeof(object_info));
info->data_size = data_size;
object_table_add_data *info = malloc(sizeof(object_table_add_data));
info->object_size = object_size;
memcpy(&info->digest[0], digest, DIGEST_SIZE);
init_table_callback(db_handle, object_id, __func__, info, retry,
done_callback, redis_object_table_add, user_context);
}

void object_table_subscribe(
void object_table_subscribe_to_notifications(
db_handle *db_handle,
object_id object_id,
object_table_object_available_callback object_available_callback,
void *subscribe_context,
retry_info *retry,
Expand All @@ -42,9 +41,25 @@ void object_table_subscribe(
sub_data->object_available_callback = object_available_callback;
sub_data->subscribe_context = subscribe_context;

init_table_callback(db_handle, object_id, __func__, sub_data, retry,
done_callback, redis_object_table_subscribe,
user_context);
init_table_callback(
db_handle, NIL_OBJECT_ID, __func__, sub_data, retry, done_callback,
redis_object_table_subscribe_to_notifications, user_context);
}

void object_table_request_notifications(db_handle *db_handle,
int num_object_ids,
object_id object_ids[],
retry_info *retry) {
CHECK(db_handle != NULL);
CHECK(num_object_ids > 0);
object_table_request_notifications_data *data =
malloc(sizeof(object_table_request_notifications_data) +
num_object_ids * sizeof(object_id));
data->num_object_ids = num_object_ids;
memcpy(data->object_ids, object_ids, num_object_ids * sizeof(object_id));

init_table_callback(db_handle, NIL_OBJECT_ID, __func__, data, retry, NULL,
redis_object_table_request_notifications, NULL);
}

void object_info_subscribe(db_handle *db_handle,
Expand Down
Loading

0 comments on commit 269f37e

Please sign in to comment.