mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Fix some bugs (#22)
* Bug fixes. * Remove plasma_reply_type. * Fix formatting. * Speed up tests a little. * Small fixes.
This commit is contained in:
parent
494903c5d4
commit
96a59200d3
4 changed files with 70 additions and 79 deletions
17
src/plasma.h
17
src/plasma.h
|
@ -25,6 +25,15 @@
|
|||
fprintf(stderr, "[ERROR] (%s:%d: errno: %s) " M "\n", __FILE__, __LINE__, \
|
||||
errno == 0 ? "None" : strerror(errno), ##__VA_ARGS__)
|
||||
|
||||
#define PLASMA_CHECK(CONDITION, M, ...) \
|
||||
do { \
|
||||
if (!(CONDITION)) { \
|
||||
fprintf(stderr, "[FATAL] (%s:%d " #CONDITION ") \n" M, __FILE__, \
|
||||
__LINE__); \
|
||||
exit(-1); \
|
||||
} \
|
||||
} while (0)
|
||||
|
||||
typedef struct {
|
||||
int64_t size;
|
||||
int64_t create_time;
|
||||
|
@ -55,15 +64,7 @@ typedef struct {
|
|||
int port;
|
||||
} plasma_request;
|
||||
|
||||
enum plasma_reply_type {
|
||||
/* the file descriptor represents an object */
|
||||
PLASMA_OBJECT,
|
||||
/* the file descriptor represents a future */
|
||||
PLASMA_FUTURE,
|
||||
};
|
||||
|
||||
typedef struct {
|
||||
int type;
|
||||
ptrdiff_t offset;
|
||||
int64_t map_size;
|
||||
int64_t object_size;
|
||||
|
|
|
@ -30,7 +30,6 @@ void plasma_create(int conn, plasma_id object_id, int64_t size, void **data) {
|
|||
plasma_send(conn, &req);
|
||||
plasma_reply reply;
|
||||
int fd = recv_fd(conn, (char *) &reply, sizeof(plasma_reply));
|
||||
assert(reply.type == PLASMA_OBJECT);
|
||||
assert(reply.object_size == size);
|
||||
*data =
|
||||
mmap(NULL, reply.map_size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0) +
|
||||
|
@ -48,12 +47,6 @@ void plasma_get(int conn, plasma_id object_id, int64_t *size, void **data) {
|
|||
plasma_reply reply;
|
||||
/* The following loop is run at most twice. */
|
||||
int fd = recv_fd(conn, (char *) &reply, sizeof(plasma_reply));
|
||||
if (reply.type == PLASMA_FUTURE) {
|
||||
int new_fd = recv_fd(fd, (char *) &reply, sizeof(plasma_reply));
|
||||
close(fd);
|
||||
fd = new_fd;
|
||||
}
|
||||
assert(reply.type == PLASMA_OBJECT);
|
||||
*data =
|
||||
mmap(NULL, reply.map_size, PROT_READ, MAP_SHARED, fd, 0) + reply.offset;
|
||||
if (*data == MAP_FAILED) {
|
||||
|
|
|
@ -27,7 +27,7 @@
|
|||
#include "plasma.h"
|
||||
#include "event_loop.h"
|
||||
|
||||
#define MAX_NUM_CLIENTS 100000
|
||||
#define MAX_NUM_CLIENTS 100
|
||||
|
||||
void* dlmalloc(size_t);
|
||||
|
||||
|
@ -81,6 +81,10 @@ object_notify_entry* objects_notify = NULL;
|
|||
void create_object(int conn, plasma_request* req) {
|
||||
LOG_INFO("creating object"); /* TODO(pcm): add object_id here */
|
||||
|
||||
object_table_entry* entry;
|
||||
HASH_FIND(handle, open_objects, &req->object_id, sizeof(plasma_id), entry);
|
||||
PLASMA_CHECK(entry == NULL, "Cannot create object twice.");
|
||||
|
||||
void* pointer = dlmalloc(req->size);
|
||||
int fd;
|
||||
int64_t map_size;
|
||||
|
@ -88,7 +92,7 @@ void create_object(int conn, plasma_request* req) {
|
|||
get_malloc_mapinfo(pointer, &fd, &map_size, &offset);
|
||||
assert(fd != -1);
|
||||
|
||||
object_table_entry* entry = malloc(sizeof(object_table_entry));
|
||||
entry = malloc(sizeof(object_table_entry));
|
||||
memcpy(&entry->object_id, &req->object_id, 20);
|
||||
entry->info.size = req->size;
|
||||
/* TODO(pcm): set the other fields */
|
||||
|
@ -98,7 +102,6 @@ void create_object(int conn, plasma_request* req) {
|
|||
HASH_ADD(handle, open_objects, object_id, sizeof(plasma_id), entry);
|
||||
plasma_reply reply;
|
||||
memset(&reply, 0, sizeof(reply));
|
||||
reply.type = PLASMA_OBJECT;
|
||||
reply.offset = offset;
|
||||
reply.map_size = map_size;
|
||||
reply.object_size = req->size;
|
||||
|
@ -110,21 +113,29 @@ void get_object(int conn, plasma_request* req) {
|
|||
object_table_entry* entry;
|
||||
HASH_FIND(handle, sealed_objects, &req->object_id, sizeof(plasma_id), entry);
|
||||
if (entry) {
|
||||
plasma_reply reply = {PLASMA_OBJECT, entry->offset, entry->map_size,
|
||||
entry->info.size};
|
||||
plasma_reply reply;
|
||||
memset(&reply, 0, sizeof(plasma_reply));
|
||||
reply.offset = entry->offset;
|
||||
reply.map_size = entry->map_size;
|
||||
reply.object_size = entry->info.size;
|
||||
send_fd(conn, entry->fd, (char*) &reply, sizeof(plasma_reply));
|
||||
} else {
|
||||
object_notify_entry* notify_entry;
|
||||
LOG_INFO("object not in hash table of sealed objects");
|
||||
int fd[2];
|
||||
socketpair(AF_UNIX, SOCK_STREAM, 0, fd);
|
||||
object_notify_entry* notify_entry = malloc(sizeof(object_notify_entry));
|
||||
memcpy(¬ify_entry->object_id, &req->object_id, 20);
|
||||
notify_entry->conn[notify_entry->num_waiting] = fd[0];
|
||||
HASH_FIND(handle, objects_notify, &req->object_id, sizeof(plasma_id),
|
||||
notify_entry);
|
||||
if (!notify_entry) {
|
||||
notify_entry = malloc(sizeof(object_notify_entry));
|
||||
memset(notify_entry, 0, sizeof(object_notify_entry));
|
||||
notify_entry->num_waiting = 0;
|
||||
memcpy(¬ify_entry->object_id, &req->object_id, 20);
|
||||
HASH_ADD(handle, objects_notify, object_id, sizeof(plasma_id),
|
||||
notify_entry);
|
||||
}
|
||||
PLASMA_CHECK(notify_entry->num_waiting < MAX_NUM_CLIENTS - 1,
|
||||
"This exceeds the maximum number of clients.");
|
||||
notify_entry->conn[notify_entry->num_waiting] = conn;
|
||||
notify_entry->num_waiting += 1;
|
||||
HASH_ADD(handle, objects_notify, object_id, sizeof(plasma_id),
|
||||
notify_entry);
|
||||
plasma_reply reply = {PLASMA_FUTURE, 0, 0, -1};
|
||||
send_fd(conn, fd[1], (char*) &reply, sizeof(plasma_reply));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -145,12 +156,12 @@ void seal_object(int conn, plasma_request* req) {
|
|||
if (!notify_entry) {
|
||||
return;
|
||||
}
|
||||
plasma_reply reply = {PLASMA_OBJECT, entry->offset, entry->map_size,
|
||||
entry->info.size};
|
||||
plasma_reply reply = {.offset = entry->offset,
|
||||
.map_size = entry->map_size,
|
||||
.object_size = entry->info.size};
|
||||
for (int i = 0; i < notify_entry->num_waiting; ++i) {
|
||||
send_fd(notify_entry->conn[i], entry->fd, (char*) &reply,
|
||||
sizeof(plasma_reply));
|
||||
close(notify_entry->conn[i]);
|
||||
}
|
||||
HASH_DELETE(handle, objects_notify, notify_entry);
|
||||
free(notify_entry);
|
||||
|
@ -189,19 +200,10 @@ void run_event_loop(int socket) {
|
|||
if (waiting->revents == 0)
|
||||
continue;
|
||||
if (waiting->fd == socket) {
|
||||
while (1) {
|
||||
/* Handle new incoming connections. */
|
||||
int new_socket = accept(socket, NULL, NULL);
|
||||
if (new_socket < 0) {
|
||||
if (errno != EWOULDBLOCK) {
|
||||
LOG_ERR("accept failed");
|
||||
exit(-1);
|
||||
}
|
||||
break;
|
||||
}
|
||||
event_loop_attach(state.loop, 0, NULL, new_socket, POLLIN);
|
||||
LOG_INFO("adding new client");
|
||||
}
|
||||
/* Handle new incoming connections. */
|
||||
int new_socket = accept(socket, NULL, NULL);
|
||||
event_loop_attach(state.loop, 0, NULL, new_socket, POLLIN);
|
||||
LOG_INFO("adding new client");
|
||||
} else {
|
||||
int r = read(waiting->fd, &req, sizeof(plasma_request));
|
||||
if (r == -1) {
|
||||
|
@ -230,12 +232,6 @@ void start_server(char* socket_name) {
|
|||
close(fd);
|
||||
exit(-1);
|
||||
}
|
||||
/* TODO(pcm): http://stackoverflow.com/q/1150635 */
|
||||
if (ioctl(fd, FIONBIO, (char*) &on) < 0) {
|
||||
LOG_ERR("ioctl failed");
|
||||
close(fd);
|
||||
exit(-1);
|
||||
}
|
||||
struct sockaddr_un addr;
|
||||
memset(&addr, 0, sizeof(addr));
|
||||
addr.sun_family = AF_UNIX;
|
||||
|
|
57
test/test.py
57
test/test.py
|
@ -94,34 +94,35 @@ class TestPlasmaManager(unittest.TestCase):
|
|||
self.p5.kill()
|
||||
|
||||
def test_transfer(self):
|
||||
# Create an object id string.
|
||||
object_id1 = random_object_id()
|
||||
# Create a new buffer and write to it.
|
||||
memory_buffer = self.client1.create(object_id1, 20000)
|
||||
for i in range(len(memory_buffer)):
|
||||
memory_buffer[i] = chr(i % 10)
|
||||
# Seal the buffer.
|
||||
self.client1.seal(object_id1)
|
||||
# Transfer the buffer to the the other PlasmaStore.
|
||||
self.client1.transfer("127.0.0.1", self.port2, object_id1)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id1)[:], self.client2.get(object_id1)[:])
|
||||
# Transfer the buffer again.
|
||||
self.client1.transfer("127.0.0.1", self.port2, object_id1)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id1)[:], self.client2.get(object_id1)[:])
|
||||
# Create a new object id string.
|
||||
object_id2 = random_object_id()
|
||||
# Create a new buffer and write to it.
|
||||
memory_buffer = self.client2.create(object_id2, 20000)
|
||||
for i in range(len(memory_buffer)):
|
||||
memory_buffer[i] = chr(i % 10)
|
||||
# Seal the buffer.
|
||||
self.client2.seal(object_id2)
|
||||
# Transfer the buffer to the the other PlasmaStore.
|
||||
self.client2.transfer("127.0.0.1", self.port1, object_id2)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id2)[:], self.client2.get(object_id2)[:])
|
||||
for _ in range(100):
|
||||
# Create an object id string.
|
||||
object_id1 = random_object_id()
|
||||
# Create a new buffer and set the first and last entries.
|
||||
memory_buffer = self.client1.create(object_id1, 20000)
|
||||
memory_buffer[0] = chr(1)
|
||||
memory_buffer[-1] = chr(2)
|
||||
# Seal the buffer.
|
||||
self.client1.seal(object_id1)
|
||||
# Transfer the buffer to the the other PlasmaStore.
|
||||
self.client1.transfer("127.0.0.1", self.port2, object_id1)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id1)[:], self.client2.get(object_id1)[:])
|
||||
# Transfer the buffer again.
|
||||
self.client1.transfer("127.0.0.1", self.port2, object_id1)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id1)[:], self.client2.get(object_id1)[:])
|
||||
# Create a new object id string.
|
||||
object_id2 = random_object_id()
|
||||
# Create a new buffer and set the first and last entries.
|
||||
memory_buffer = self.client2.create(object_id2, 20000)
|
||||
memory_buffer[0] = chr(3)
|
||||
memory_buffer[-1] = chr(4)
|
||||
# Seal the buffer.
|
||||
self.client2.seal(object_id2)
|
||||
# Transfer the buffer to the the other PlasmaStore.
|
||||
self.client2.transfer("127.0.0.1", self.port1, object_id2)
|
||||
# Compare the two buffers.
|
||||
self.assertEqual(self.client1.get(object_id2)[:], self.client2.get(object_id2)[:])
|
||||
|
||||
def test_illegal_functionality(self):
|
||||
# Create an object id string.
|
||||
|
|
Loading…
Add table
Reference in a new issue