Skip to content

Commit

Permalink
Fix stack overflow if many objects are fetched. (ray-project#237)
Browse files Browse the repository at this point in the history
* fix stack overflow if many objects are fetched

* fix other stack allocations

* add tests and fix linting

* address stephanie's comments

* fix linting

* fix tests
  • Loading branch information
pcmoritz authored and robertnishihara committed Feb 5, 2017
1 parent e5a9fc0 commit ca254b8
Show file tree
Hide file tree
Showing 5 changed files with 34 additions and 7 deletions.
7 changes: 5 additions & 2 deletions src/numbuf/python/src/pynumbuf/numbuf.cc
Original file line number Diff line number Diff line change
Expand Up @@ -320,8 +320,8 @@ static PyObject* retrieve_list(PyObject* self, PyObject* args) {
if (!PyObjectToPlasmaConnection(plasma_conn, &conn)) { return NULL; }

Py_ssize_t num_object_ids = PyList_Size(object_id_list);
object_id object_ids[num_object_ids];
object_buffer object_buffers[num_object_ids];
object_id* object_ids = new object_id[num_object_ids];
object_buffer* object_buffers = new object_buffer[num_object_ids];

for (int i = 0; i < num_object_ids; ++i) {
PyStringToUniqueID(PyList_GetItem(object_id_list, i), &object_ids[i]);
Expand Down Expand Up @@ -374,6 +374,9 @@ static PyObject* retrieve_list(PyObject* self, PyObject* args) {
PyList_SetItem(returns, i, t);
}

delete[] object_ids;
delete[] object_buffers;

return returns;
}

Expand Down
6 changes: 4 additions & 2 deletions src/plasma/plasma_client.c
Original file line number Diff line number Diff line change
Expand Up @@ -316,8 +316,8 @@ void plasma_get(plasma_connection *conn,
num_objects, timeout_ms) >= 0);
uint8_t *reply_data =
plasma_receive(conn->store_conn, MessageType_PlasmaGetReply);
object_id received_obj_ids[num_objects];
plasma_object object_data[num_objects];
object_id *received_obj_ids = malloc(num_objects * sizeof(object_id));
plasma_object *object_data = malloc(num_objects * sizeof(plasma_object));
plasma_object *object;
plasma_read_GetReply(reply_data, received_obj_ids, object_data, num_objects);
free(reply_data);
Expand Down Expand Up @@ -365,6 +365,8 @@ void plasma_get(plasma_connection *conn,
object_buffers[i].data_size = -1;
}
}
free(object_data);
free(received_obj_ids);
}

/**
Expand Down
11 changes: 9 additions & 2 deletions src/plasma/plasma_manager.c
Original file line number Diff line number Diff line change
Expand Up @@ -1362,20 +1362,27 @@ void process_message(event_loop *loop,
case MessageType_PlasmaFetchRequest: {
LOG_DEBUG("Processing fetch remote");
int64_t num_objects = plasma_read_FetchRequest_num_objects(data);
object_id object_ids_to_fetch[num_objects];
object_id *object_ids_to_fetch = malloc(num_objects * sizeof(object_id));
/* TODO(pcm): process_fetch_requests allocates an array of num_objects
* object_ids too so these should be shared in the future. */
plasma_read_FetchRequest(data, object_ids_to_fetch, num_objects);
process_fetch_requests(conn, num_objects, &object_ids_to_fetch[0]);
free(object_ids_to_fetch);
} break;
case MessageType_PlasmaWaitRequest: {
LOG_DEBUG("Processing wait");
int num_object_ids = plasma_read_WaitRequest_num_object_ids(data);
object_request *object_requests =
malloc(num_object_ids * sizeof(object_request));
int64_t timeout_ms;
int num_ready_objects;
object_request object_requests[num_object_ids];
plasma_read_WaitRequest(data, &object_requests[0], num_object_ids,
&timeout_ms, &num_ready_objects);
/* TODO(pcm): process_wait_requests allocates an array of num_object_ids
* object_requests too so these could be shared in the future. */
process_wait_request(conn, num_object_ids, &object_requests[0], timeout_ms,
num_ready_objects);
free(object_requests);
} break;
case MessageType_PlasmaStatusRequest: {
LOG_DEBUG("Processing status");
Expand Down
5 changes: 4 additions & 1 deletion src/plasma/plasma_store.c
Original file line number Diff line number Diff line change
Expand Up @@ -737,11 +737,14 @@ void process_message(event_loop *loop,
} break;
case MessageType_PlasmaGetRequest: {
num_objects = plasma_read_GetRequest_num_objects(input);
object_id object_ids_to_get[num_objects];
object_id *object_ids_to_get = malloc(num_objects * sizeof(object_id));
int64_t timeout_ms;
plasma_read_GetRequest(input, object_ids_to_get, &timeout_ms, num_objects);
/* TODO(pcm): The array object_ids_to_get could be reused in
* process_get_request. */
process_get_request(client_context, num_objects, object_ids_to_get,
timeout_ms);
free(object_ids_to_get);
} break;
case MessageType_PlasmaReleaseRequest:
plasma_read_ReleaseRequest(input, &object_ids[0]);
Expand Down
12 changes: 12 additions & 0 deletions test/stress_tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,18 @@ def testGettingAndPutting(self):
self.assertTrue(ray.services.all_processes_alive())
ray.worker.cleanup()

def testGettingManyObjects(self):
ray.init()

@ray.remote
def f():
return 1

n = 10 ** 4 # TODO(pcm): replace by 10 ** 5 once this is faster
l = ray.get([f.remote() for _ in range(n)])
self.assertEqual(l, n * [1])
ray.worker.cleanup()

def testWait(self):
for num_local_schedulers in [1, 4]:
for num_workers_per_scheduler in [4]:
Expand Down

0 comments on commit ca254b8

Please sign in to comment.