Skip to content

Commit cff3776

Browse files
elibolpcmoritz
authored andcommitted
Addresses missed comments from multichunk object transfer PR. (#1908)
* Move object manager parameters to ray config, object manager config bug fix. addresses other comments from #1827. * linting and uint? * typos * remove uint.
1 parent 6ca2c2a commit cff3776

File tree

9 files changed

+105
-37
lines changed

9 files changed

+105
-37
lines changed

src/common/state/ray_config.h

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -86,6 +86,20 @@ class RayConfig {
8686
return actor_creation_num_spillbacks_warning_;
8787
}
8888

89+
int object_manager_pull_timeout_ms() const {
90+
return object_manager_pull_timeout_ms_;
91+
}
92+
93+
int object_manager_max_sends() const { return object_manager_max_sends_; }
94+
95+
int object_manager_max_receives() const {
96+
return object_manager_max_receives_;
97+
}
98+
99+
uint64_t object_manager_default_chunk_size() const {
100+
return object_manager_default_chunk_size_;
101+
}
102+
89103
private:
90104
RayConfig()
91105
: ray_protocol_version_(0x0000000000000000),
@@ -113,7 +127,14 @@ class RayConfig {
113127
plasma_default_release_delay_(64),
114128
L3_cache_size_bytes_(100000000),
115129
max_tasks_to_spillback_(10),
116-
actor_creation_num_spillbacks_warning_(100) {}
130+
actor_creation_num_spillbacks_warning_(100),
131+
// TODO: Setting this to large values results in latency, which needs to
132+
// be addressed. This timeout is often on the critical path for object
133+
// transfers.
134+
object_manager_pull_timeout_ms_(20),
135+
object_manager_max_sends_(2),
136+
object_manager_max_receives_(2),
137+
object_manager_default_chunk_size_(100000000) {}
117138

118139
~RayConfig() {}
119140

@@ -196,6 +217,21 @@ class RayConfig {
196217
/// corresponding driver. Since spillback currently occurs on a 100ms timer,
197218
/// a value of 100 corresponds to a warning every 10 seconds.
198219
int64_t actor_creation_num_spillbacks_warning_;
220+
221+
/// Timeout, in milliseconds, to wait before retrying a failed pull in the
222+
/// ObjectManager.
223+
int object_manager_pull_timeout_ms_;
224+
225+
/// Maximum number of concurrent sends allowed by the object manager.
226+
int object_manager_max_sends_;
227+
228+
/// Maximum number of concurrent receives allowed by the object manager.
229+
int object_manager_max_receives_;
230+
231+
/// Default chunk size for multi-chunk transfers to use in the object manager.
232+
/// In the object manager, no single thread is permitted to transfer more
233+
/// data than what is specified by the chunk size.
234+
uint64_t object_manager_default_chunk_size_;
199235
};
200236

201237
#endif // RAY_CONFIG_H

src/ray/object_manager/object_manager.cc

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -12,12 +12,13 @@ ObjectManager::ObjectManager(asio::io_service &main_service,
1212
std::shared_ptr<gcs::AsyncGcsClient> gcs_client)
1313
// TODO(hme): Eliminate knowledge of GCS.
1414
: client_id_(gcs_client->client_table().GetLocalClientId()),
15+
config_(config),
1516
object_directory_(new ObjectDirectory(gcs_client)),
16-
store_notification_(main_service, config.store_socket_name),
17-
// release_delay of 2 * config.max_sends is to ensure the pool does not release
17+
store_notification_(main_service, config_.store_socket_name),
18+
// release_delay of 2 * config_.max_sends is to ensure the pool does not release
1819
// an object prematurely whenever we reach the maximum number of sends.
19-
buffer_pool_(config.store_socket_name, config.object_chunk_size,
20-
/*release_delay=*/2 * config.max_sends),
20+
buffer_pool_(config_.store_socket_name, config_.object_chunk_size,
21+
/*release_delay=*/2 * config_.max_sends),
2122
object_manager_service_(std::move(object_manager_service)),
2223
work_(*object_manager_service_),
2324
connection_pool_(),
@@ -28,7 +29,6 @@ ObjectManager::ObjectManager(asio::io_service &main_service,
2829
RAY_CHECK(config_.max_sends > 0);
2930
RAY_CHECK(config_.max_receives > 0);
3031
main_service_ = &main_service;
31-
config_ = config;
3232
store_notification_.SubscribeObjAdded(
3333
[this](const ObjectInfoT &object_info) { NotifyDirectoryObjectAdd(object_info); });
3434
store_notification_.SubscribeObjDeleted(
@@ -40,12 +40,13 @@ ObjectManager::ObjectManager(asio::io_service &main_service,
4040
std::unique_ptr<asio::io_service> object_manager_service,
4141
const ObjectManagerConfig &config,
4242
std::unique_ptr<ObjectDirectoryInterface> od)
43-
: object_directory_(std::move(od)),
44-
store_notification_(main_service, config.store_socket_name),
45-
// release_delay of 2 * config.max_sends is to ensure the pool does not release
43+
: config_(config),
44+
object_directory_(std::move(od)),
45+
store_notification_(main_service, config_.store_socket_name),
46+
// release_delay of 2 * config_.max_sends is to ensure the pool does not release
4647
// an object prematurely whenever we reach the maximum number of sends.
47-
buffer_pool_(config.store_socket_name, config.object_chunk_size,
48-
/*release_delay=*/2 * config.max_sends),
48+
buffer_pool_(config_.store_socket_name, config_.object_chunk_size,
49+
/*release_delay=*/2 * config_.max_sends),
4950
object_manager_service_(std::move(object_manager_service)),
5051
work_(*object_manager_service_),
5152
connection_pool_(),
@@ -57,7 +58,6 @@ ObjectManager::ObjectManager(asio::io_service &main_service,
5758
RAY_CHECK(config_.max_receives > 0);
5859
// TODO(hme) Client ID is never set with this constructor.
5960
main_service_ = &main_service;
60-
config_ = config;
6161
store_notification_.SubscribeObjAdded(
6262
[this](const ObjectInfoT &object_info) { NotifyDirectoryObjectAdd(object_info); });
6363
store_notification_.SubscribeObjDeleted(

src/ray/object_manager/object_manager.h

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -33,13 +33,13 @@ namespace ray {
3333
struct ObjectManagerConfig {
3434
/// The time in milliseconds to wait before retrying a pull
3535
/// that failed due to client id lookup.
36-
uint pull_timeout_ms = 100;
36+
uint pull_timeout_ms;
3737
/// Maximum number of sends allowed.
38-
int max_sends = 2;
38+
int max_sends;
3939
/// Maximum number of receives allowed.
40-
int max_receives = 2;
40+
int max_receives;
4141
/// Object chunk size, in bytes
42-
uint64_t object_chunk_size = std::pow(10, 8);
42+
uint64_t object_chunk_size;
4343
// TODO(hme): Implement num retries (to avoid infinite retries).
4444
std::string store_socket_name;
4545
};
@@ -152,7 +152,7 @@ class ObjectManager {
152152

153153
private:
154154
ClientID client_id_;
155-
ObjectManagerConfig config_;
155+
const ObjectManagerConfig config_;
156156
std::unique_ptr<ObjectDirectoryInterface> object_directory_;
157157
ObjectStoreNotificationManager store_notification_;
158158
ObjectBufferPool buffer_pool_;

src/ray/object_manager/test/object_manager_stress_test.cc

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -125,23 +125,30 @@ class TestObjectManagerBase : public ::testing::Test {
125125
store_id_1 = StartStore(UniqueID::from_random().hex());
126126
store_id_2 = StartStore(UniqueID::from_random().hex());
127127

128+
uint pull_timeout_ms = 1;
129+
int max_sends = 2;
130+
int max_receives = 2;
131+
uint64_t object_chunk_size = static_cast<uint64_t>(std::pow(10, 3));
132+
128133
// start first server
129134
gcs_client_1 = std::shared_ptr<gcs::AsyncGcsClient>(new gcs::AsyncGcsClient());
130135
ObjectManagerConfig om_config_1;
131136
om_config_1.store_socket_name = store_id_1;
132-
om_config_1.max_sends = 2;
133-
om_config_1.max_receives = 2;
134-
om_config_1.object_chunk_size = static_cast<uint64_t>(std::pow(10, 3));
137+
om_config_1.pull_timeout_ms = pull_timeout_ms;
138+
om_config_1.max_sends = max_sends;
139+
om_config_1.max_receives = max_receives;
140+
om_config_1.object_chunk_size = object_chunk_size;
135141
server1.reset(new MockServer(main_service, std::move(object_manager_service_1),
136142
om_config_1, gcs_client_1));
137143

138144
// start second server
139145
gcs_client_2 = std::shared_ptr<gcs::AsyncGcsClient>(new gcs::AsyncGcsClient());
140146
ObjectManagerConfig om_config_2;
141147
om_config_2.store_socket_name = store_id_2;
142-
om_config_2.max_sends = 2;
143-
om_config_2.max_receives = 2;
144-
om_config_2.object_chunk_size = static_cast<uint64_t>(std::pow(10, 3));
148+
om_config_2.pull_timeout_ms = pull_timeout_ms;
149+
om_config_2.max_sends = max_sends;
150+
om_config_2.max_receives = max_receives;
151+
om_config_2.object_chunk_size = object_chunk_size;
145152
server2.reset(new MockServer(main_service, std::move(object_manager_service_2),
146153
om_config_2, gcs_client_2));
147154

src/ray/object_manager/test/object_manager_test.cc

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -115,17 +115,30 @@ class TestObjectManager : public ::testing::Test {
115115
store_id_1 = StartStore(UniqueID::from_random().hex());
116116
store_id_2 = StartStore(UniqueID::from_random().hex());
117117

118+
uint pull_timeout_ms = 1;
119+
int max_sends = 2;
120+
int max_receives = 2;
121+
uint64_t object_chunk_size = static_cast<uint64_t>(std::pow(10, 3));
122+
118123
// start first server
119124
gcs_client_1 = std::shared_ptr<gcs::AsyncGcsClient>(new gcs::AsyncGcsClient());
120125
ObjectManagerConfig om_config_1;
121126
om_config_1.store_socket_name = store_id_1;
127+
om_config_1.pull_timeout_ms = pull_timeout_ms;
128+
om_config_1.max_sends = max_sends;
129+
om_config_1.max_receives = max_receives;
130+
om_config_1.object_chunk_size = object_chunk_size;
122131
server1.reset(new MockServer(main_service, std::move(object_manager_service_1),
123132
om_config_1, gcs_client_1));
124133

125134
// start second server
126135
gcs_client_2 = std::shared_ptr<gcs::AsyncGcsClient>(new gcs::AsyncGcsClient());
127136
ObjectManagerConfig om_config_2;
128137
om_config_2.store_socket_name = store_id_2;
138+
om_config_2.pull_timeout_ms = pull_timeout_ms;
139+
om_config_2.max_sends = max_sends;
140+
om_config_2.max_receives = max_receives;
141+
om_config_2.object_chunk_size = object_chunk_size;
129142
server2.reset(new MockServer(main_service, std::move(object_manager_service_2),
130143
om_config_2, gcs_client_2));
131144

src/ray/object_manager/transfer_queue.cc

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@ namespace ray {
55
void TransferQueue::QueueSend(const ClientID &client_id, const ObjectID &object_id,
66
uint64_t data_size, uint64_t metadata_size,
77
uint64_t chunk_index, const RemoteConnectionInfo &info) {
8-
std::unique_lock<std::mutex> guard(send_mutex_);
8+
std::lock_guard<std::mutex> guard(send_mutex_);
99
SendRequest req = {client_id, object_id, data_size, metadata_size, chunk_index, info};
1010
// TODO(hme): Use a set to speed this up.
1111
if (std::find(send_queue_.begin(), send_queue_.end(), req) != send_queue_.end()) {
@@ -19,7 +19,7 @@ void TransferQueue::QueueReceive(const ClientID &client_id, const ObjectID &obje
1919
uint64_t data_size, uint64_t metadata_size,
2020
uint64_t chunk_index,
2121
std::shared_ptr<TcpClientConnection> conn) {
22-
std::unique_lock<std::mutex> guard(receive_mutex_);
22+
std::lock_guard<std::mutex> guard(receive_mutex_);
2323
ReceiveRequest req = {client_id, object_id, data_size,
2424
metadata_size, chunk_index, conn};
2525
if (std::find(receive_queue_.begin(), receive_queue_.end(), req) !=
@@ -31,7 +31,7 @@ void TransferQueue::QueueReceive(const ClientID &client_id, const ObjectID &obje
3131
}
3232

3333
bool TransferQueue::DequeueSendIfPresent(TransferQueue::SendRequest *send_ptr) {
34-
std::unique_lock<std::mutex> guard(send_mutex_);
34+
std::lock_guard<std::mutex> guard(send_mutex_);
3535
if (send_queue_.empty()) {
3636
return false;
3737
}
@@ -41,7 +41,7 @@ bool TransferQueue::DequeueSendIfPresent(TransferQueue::SendRequest *send_ptr) {
4141
}
4242

4343
bool TransferQueue::DequeueReceiveIfPresent(TransferQueue::ReceiveRequest *receive_ptr) {
44-
std::unique_lock<std::mutex> guard(receive_mutex_);
44+
std::lock_guard<std::mutex> guard(receive_mutex_);
4545
if (receive_queue_.empty()) {
4646
return false;
4747
}

src/ray/object_manager/transfer_queue.h

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,12 @@ class TransferQueue {
6161
///
6262
/// \param client_id The ClientID to which the object needs to be sent.
6363
/// \param object_id The ObjectID of the object to be sent.
64+
/// \param data_size The actual object size + the metadata size.
65+
/// \param metadata_size The size of the object's metadata.
66+
/// \param chunk_index The chunk index, which corresponds to the chunk of object_id that
67+
/// is queued for transfer.
68+
/// \param info Connection information to the remote node, which is required if a new
69+
/// connection needs to be established.
6470
void QueueSend(const ClientID &client_id, const ObjectID &object_id, uint64_t data_size,
6571
uint64_t metadata_size, uint64_t chunk_index,
6672
const RemoteConnectionInfo &info);
@@ -77,6 +83,11 @@ class TransferQueue {
7783
///
7884
/// \param client_id The ClientID from which the object is being received.
7985
/// \param object_id The ObjectID of the object to be received.
86+
/// \param data_size The actual object size + the metadata size.
87+
/// \param metadata_size The size of the object's metadata.
88+
/// \param chunk_index The chunk index, which corresponds to the chunk of object_id that
89+
/// is queued for transfer.
90+
/// \param conn Connection to the remote object manager that's sending data.
8091
void QueueReceive(const ClientID &client_id, const ObjectID &object_id,
8192
uint64_t data_size, uint64_t metadata_size, uint64_t chunk_index,
8293
std::shared_ptr<TcpClientConnection> conn);
@@ -93,7 +104,9 @@ class TransferQueue {
93104
RAY_DISALLOW_COPY_AND_ASSIGN(TransferQueue);
94105

95106
private:
107+
/// Locks access to send_queue_.
96108
std::mutex send_mutex_;
109+
/// Locks access to receive_queue_.
97110
std::mutex receive_mutex_;
98111
std::deque<SendRequest> send_queue_;
99112
std::deque<ReceiveRequest> receive_queue_;

src/ray/raylet/main.cc

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
#include <iostream>
22

3+
#include "common/state/ray_config.h"
34
#include "ray/raylet/raylet.h"
45
#include "ray/status.h"
56

@@ -47,14 +48,13 @@ int main(int argc, char *argv[]) {
4748
// Configuration for the object manager.
4849
ray::ObjectManagerConfig object_manager_config;
4950
object_manager_config.store_socket_name = store_socket_name;
50-
// Time out in milliseconds to wait before retrying a failed pull.
51-
object_manager_config.pull_timeout_ms = 100;
52-
// Maximum number of sends allowed.
53-
object_manager_config.max_sends = 2;
54-
// Maximum number of receives allowed.
55-
object_manager_config.max_receives = 2;
56-
// Object chunk size, in bytes.
57-
object_manager_config.object_chunk_size = static_cast<uint64_t>(std::pow(10, 8));
51+
object_manager_config.pull_timeout_ms =
52+
RayConfig::instance().object_manager_pull_timeout_ms();
53+
object_manager_config.max_sends = RayConfig::instance().object_manager_max_sends();
54+
object_manager_config.max_receives =
55+
RayConfig::instance().object_manager_max_receives();
56+
object_manager_config.object_chunk_size =
57+
RayConfig::instance().object_manager_default_chunk_size();
5858

5959
// initialize mock gcs & object directory
6060
auto gcs_client = std::make_shared<ray::gcs::AsyncGcsClient>();

src/ray/test/run_object_manager_tests.sh

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,8 +30,7 @@ echo "$REDIS_DIR/redis-server --loglevel warning --loadmodule $REDIS_MODULE --po
3030
echo "$REDIS_DIR/redis-cli -p 6379 shutdown"
3131

3232
# Allow cleanup commands to fail.
33-
# killall plasma_store || true
34-
# $REDIS_DIR/redis-cli -p 6379 shutdown || true
33+
$REDIS_DIR/redis-cli -p 6379 shutdown || true
3534
sleep 1s
3635
$REDIS_DIR/redis-server --loglevel warning --loadmodule $REDIS_MODULE --port 6379 &
3736
sleep 1s

0 commit comments

Comments
 (0)