Skip to content

Commit 5b45f0b

Browse files
heyucongtomrobertnishihara
authored andcommitted
[xray] Implementing Gcs sharding (#2409)
Basically a re-implementation of #2281, with modifications of #2298 (A fix of #2334, for rebasing issues.). [+] Implement sharding for gcs tables. [+] Keep ClientTable and ErrorTable managed by the primary_shard. TaskTable is managed by the primary_shard for now, until a good hashing for tasks is implemented. [+] Move AsyncGcsClient's initialization into Connect function. [-] Move GetRedisShard and bool sharding from RedisContext's connect into AsyncGcsClient. This may make the interface cleaner.
1 parent eda6ebb commit 5b45f0b

19 files changed

+268
-234
lines changed

src/global_scheduler/global_scheduler.cc

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -132,11 +132,6 @@ GlobalSchedulerState *GlobalSchedulerState_init(event_loop *loop,
132132
"global_scheduler", node_ip_address,
133133
std::vector<std::string>());
134134
db_attach(state->db, loop, false);
135-
136-
RAY_CHECK_OK(state->gcs_client.Connect(
137-
std::string(redis_primary_addr), redis_primary_port, /*sharding=*/true));
138-
RAY_CHECK_OK(state->gcs_client.context()->AttachToEventLoop(loop));
139-
RAY_CHECK_OK(state->gcs_client.primary_context()->AttachToEventLoop(loop));
140135
state->policy_state = GlobalSchedulerPolicyState_init();
141136
return state;
142137
}

src/global_scheduler/global_scheduler.h

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -51,8 +51,6 @@ typedef struct {
5151
event_loop *loop;
5252
/** The global state store database. */
5353
DBHandle *db;
54-
/** The handle to the GCS (modern version of the above). */
55-
ray::gcs::AsyncGcsClient gcs_client;
5654
/** A hash table mapping local scheduler ID to the local schedulers that are
5755
* connected to Redis. */
5856
std::unordered_map<DBClientID, LocalScheduler> local_schedulers;

src/local_scheduler/local_scheduler.cc

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -351,11 +351,6 @@ LocalSchedulerState *LocalSchedulerState_init(
351351
state->db = db_connect(std::string(redis_primary_addr), redis_primary_port,
352352
"local_scheduler", node_ip_address, db_connect_args);
353353
db_attach(state->db, loop, false);
354-
355-
RAY_CHECK_OK(state->gcs_client.Connect(std::string(redis_primary_addr),
356-
redis_primary_port, true));
357-
RAY_CHECK_OK(state->gcs_client.context()->AttachToEventLoop(loop));
358-
RAY_CHECK_OK(state->gcs_client.primary_context()->AttachToEventLoop(loop));
359354
} else {
360355
state->db = NULL;
361356
}

src/local_scheduler/local_scheduler_shared.h

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -60,8 +60,6 @@ struct LocalSchedulerState {
6060
std::unordered_map<ActorID, ActorMapEntry> actor_mapping;
6161
/** The handle to the database. */
6262
DBHandle *db;
63-
/** The handle to the GCS (modern version of the above). */
64-
ray::gcs::AsyncGcsClient gcs_client;
6563
/** The Plasma client. */
6664
plasma::PlasmaClient *plasma_conn;
6765
/** State for the scheduling algorithm. */

src/plasma/plasma_manager.cc

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -215,8 +215,6 @@ struct PlasmaManagerState {
215215
* other plasma stores. */
216216
std::unordered_map<std::string, ClientConnection *> manager_connections;
217217
DBHandle *db;
218-
/** The handle to the GCS (modern version of the above). */
219-
ray::gcs::AsyncGcsClient gcs_client;
220218
/** Our address. */
221219
const char *addr;
222220
/** Our port. */
@@ -490,13 +488,6 @@ PlasmaManagerState *PlasmaManagerState_init(const char *store_socket_name,
490488
state->db = db_connect(std::string(redis_primary_addr), redis_primary_port,
491489
"plasma_manager", manager_addr, db_connect_args);
492490
db_attach(state->db, state->loop, false);
493-
494-
RAY_CHECK_OK(state->gcs_client.Connect(std::string(redis_primary_addr),
495-
redis_primary_port,
496-
/*sharding=*/true));
497-
RAY_CHECK_OK(state->gcs_client.context()->AttachToEventLoop(state->loop));
498-
RAY_CHECK_OK(
499-
state->gcs_client.primary_context()->AttachToEventLoop(state->loop));
500491
} else {
501492
state->db = NULL;
502493
RAY_LOG(DEBUG) << "No db connection specified";

src/ray/gcs/client.cc

Lines changed: 138 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -2,51 +2,152 @@
22

33
#include "ray/gcs/redis_context.h"
44

5+
static void GetRedisShards(redisContext *context, std::vector<std::string> &addresses,
6+
std::vector<int> &ports) {
7+
// Get the total number of Redis shards in the system.
8+
int num_attempts = 0;
9+
redisReply *reply = nullptr;
10+
while (num_attempts < RayConfig::instance().redis_db_connect_retries()) {
11+
// Try to read the number of Redis shards from the primary shard. If the
12+
// entry is present, exit.
13+
reply = reinterpret_cast<redisReply *>(redisCommand(context, "GET NumRedisShards"));
14+
if (reply->type != REDIS_REPLY_NIL) {
15+
break;
16+
}
17+
18+
// Sleep for a little, and try again if the entry isn't there yet. */
19+
freeReplyObject(reply);
20+
usleep(RayConfig::instance().redis_db_connect_wait_milliseconds() * 1000);
21+
num_attempts++;
22+
}
23+
RAY_CHECK(num_attempts < RayConfig::instance().redis_db_connect_retries())
24+
<< "No entry found for NumRedisShards";
25+
RAY_CHECK(reply->type == REDIS_REPLY_STRING) << "Expected string, found Redis type "
26+
<< reply->type << " for NumRedisShards";
27+
int num_redis_shards = atoi(reply->str);
28+
RAY_CHECK(num_redis_shards >= 1) << "Expected at least one Redis shard, "
29+
<< "found " << num_redis_shards;
30+
freeReplyObject(reply);
31+
32+
// Get the addresses of all of the Redis shards.
33+
num_attempts = 0;
34+
while (num_attempts < RayConfig::instance().redis_db_connect_retries()) {
35+
// Try to read the Redis shard locations from the primary shard. If we find
36+
// that all of them are present, exit.
37+
reply =
38+
reinterpret_cast<redisReply *>(redisCommand(context, "LRANGE RedisShards 0 -1"));
39+
if (static_cast<int>(reply->elements) == num_redis_shards) {
40+
break;
41+
}
42+
43+
// Sleep for a little, and try again if not all Redis shard addresses have
44+
// been added yet.
45+
freeReplyObject(reply);
46+
usleep(RayConfig::instance().redis_db_connect_wait_milliseconds() * 1000);
47+
num_attempts++;
48+
}
49+
RAY_CHECK(num_attempts < RayConfig::instance().redis_db_connect_retries())
50+
<< "Expected " << num_redis_shards << " Redis shard addresses, found "
51+
<< reply->elements;
52+
53+
// Parse the Redis shard addresses.
54+
for (size_t i = 0; i < reply->elements; ++i) {
55+
// Parse the shard addresses and ports.
56+
RAY_CHECK(reply->element[i]->type == REDIS_REPLY_STRING);
57+
std::string addr;
58+
std::stringstream ss(reply->element[i]->str);
59+
getline(ss, addr, ':');
60+
addresses.push_back(addr);
61+
int port;
62+
ss >> port;
63+
ports.push_back(port);
64+
}
65+
freeReplyObject(reply);
66+
}
67+
568
namespace ray {
669

770
namespace gcs {
871

9-
AsyncGcsClient::AsyncGcsClient(const ClientID &client_id, CommandType command_type) {
10-
context_ = std::make_shared<RedisContext>();
72+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port,
73+
const ClientID &client_id, CommandType command_type,
74+
bool is_test_client = false) {
1175
primary_context_ = std::make_shared<RedisContext>();
12-
client_table_.reset(new ClientTable(primary_context_, this, client_id));
13-
object_table_.reset(new ObjectTable(context_, this, command_type));
14-
actor_table_.reset(new ActorTable(context_, this));
15-
task_table_.reset(new TaskTable(context_, this, command_type));
16-
raylet_task_table_.reset(new raylet::TaskTable(context_, this, command_type));
17-
task_reconstruction_log_.reset(new TaskReconstructionLog(context_, this));
18-
task_lease_table_.reset(new TaskLeaseTable(context_, this));
19-
heartbeat_table_.reset(new HeartbeatTable(context_, this));
20-
driver_table_.reset(new DriverTable(primary_context_, this));
21-
error_table_.reset(new ErrorTable(primary_context_, this));
22-
profile_table_.reset(new ProfileTable(context_, this));
76+
77+
RAY_CHECK_OK(primary_context_->Connect(address, port, /*sharding=*/true));
78+
79+
if (!is_test_client) {
80+
// Moving sharding into constructor defaultly means that sharding = true.
81+
// This design decision may worth a look.
82+
std::vector<std::string> addresses;
83+
std::vector<int> ports;
84+
GetRedisShards(primary_context_->sync_context(), addresses, ports);
85+
if (addresses.size() == 0 || ports.size() == 0) {
86+
addresses.push_back(address);
87+
ports.push_back(port);
88+
}
89+
90+
// Populate shard_contexts.
91+
for (size_t i = 0; i < addresses.size(); ++i) {
92+
shard_contexts_.push_back(std::make_shared<RedisContext>());
93+
}
94+
95+
RAY_CHECK(shard_contexts_.size() == addresses.size());
96+
for (size_t i = 0; i < addresses.size(); ++i) {
97+
RAY_CHECK_OK(
98+
shard_contexts_[i]->Connect(addresses[i], ports[i], /*sharding=*/true));
99+
}
100+
} else {
101+
shard_contexts_.push_back(std::make_shared<RedisContext>());
102+
RAY_CHECK_OK(shard_contexts_[0]->Connect(address, port, /*sharding=*/true));
103+
}
104+
105+
client_table_.reset(new ClientTable({primary_context_}, this, client_id));
106+
error_table_.reset(new ErrorTable({primary_context_}, this));
107+
driver_table_.reset(new DriverTable({primary_context_}, this));
108+
// Tables below would be sharded.
109+
object_table_.reset(new ObjectTable(shard_contexts_, this, command_type));
110+
actor_table_.reset(new ActorTable(shard_contexts_, this));
111+
task_table_.reset(new TaskTable(shard_contexts_, this, command_type));
112+
raylet_task_table_.reset(new raylet::TaskTable(shard_contexts_, this, command_type));
113+
task_reconstruction_log_.reset(new TaskReconstructionLog(shard_contexts_, this));
114+
task_lease_table_.reset(new TaskLeaseTable(shard_contexts_, this));
115+
heartbeat_table_.reset(new HeartbeatTable(shard_contexts_, this));
116+
profile_table_.reset(new ProfileTable(shard_contexts_, this));
23117
command_type_ = command_type;
118+
119+
// TODO(swang): Call the client table's Connect() method here. To do this,
120+
// we need to make sure that we are attached to an event loop first. This
121+
// currently isn't possible because the aeEventLoop, which we use for
122+
// testing, requires us to connect to Redis first.
24123
}
25124

26125
#if RAY_USE_NEW_GCS
27126
// Use of kChain currently only applies to Table::Add which affects only the
28127
// task table, and when RAY_USE_NEW_GCS is set at compile time.
29-
AsyncGcsClient::AsyncGcsClient(const ClientID &client_id)
30-
: AsyncGcsClient(client_id, CommandType::kChain) {}
128+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port,
129+
const ClientID &client_id, bool is_test_client = false)
130+
: AsyncGcsClient(address, port, client_id, CommandType::kChain, is_test_client) {}
31131
#else
32-
AsyncGcsClient::AsyncGcsClient(const ClientID &client_id)
33-
: AsyncGcsClient(client_id, CommandType::kRegular) {}
132+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port,
133+
const ClientID &client_id, bool is_test_client = false)
134+
: AsyncGcsClient(address, port, client_id, CommandType::kRegular, is_test_client) {}
34135
#endif // RAY_USE_NEW_GCS
35136

36-
AsyncGcsClient::AsyncGcsClient(CommandType command_type)
37-
: AsyncGcsClient(ClientID::from_random(), command_type) {}
137+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port,
138+
CommandType command_type)
139+
: AsyncGcsClient(address, port, ClientID::from_random(), command_type) {}
38140

39-
AsyncGcsClient::AsyncGcsClient() : AsyncGcsClient(ClientID::from_random()) {}
141+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port,
142+
CommandType command_type, bool is_test_client)
143+
: AsyncGcsClient(address, port, ClientID::from_random(), command_type,
144+
is_test_client) {}
40145

41-
Status AsyncGcsClient::Connect(const std::string &address, int port, bool sharding) {
42-
RAY_RETURN_NOT_OK(context_->Connect(address, port, sharding));
43-
RAY_RETURN_NOT_OK(primary_context_->Connect(address, port, /*sharding=*/false));
44-
// TODO(swang): Call the client table's Connect() method here. To do this,
45-
// we need to make sure that we are attached to an event loop first. This
46-
// currently isn't possible because the aeEventLoop, which we use for
47-
// testing, requires us to connect to Redis first.
48-
return Status::OK();
49-
}
146+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port)
147+
: AsyncGcsClient(address, port, ClientID::from_random()) {}
148+
149+
AsyncGcsClient::AsyncGcsClient(const std::string &address, int port, bool is_test_client)
150+
: AsyncGcsClient(address, port, ClientID::from_random(), is_test_client) {}
50151

51152
Status Attach(plasma::EventLoop &event_loop) {
52153
// TODO(pcm): Implement this via
@@ -55,9 +156,14 @@ Status Attach(plasma::EventLoop &event_loop) {
55156
}
56157

57158
Status AsyncGcsClient::Attach(boost::asio::io_service &io_service) {
58-
asio_async_client_.reset(new RedisAsioClient(io_service, context_->async_context()));
59-
asio_subscribe_client_.reset(
60-
new RedisAsioClient(io_service, context_->subscribe_context()));
159+
// Take care of sharding contexts.
160+
RAY_CHECK(shard_asio_async_clients_.empty()) << "Attach shall be called only once";
161+
for (std::shared_ptr<RedisContext> context : shard_contexts_) {
162+
shard_asio_async_clients_.emplace_back(
163+
new RedisAsioClient(io_service, context->async_context()));
164+
shard_asio_subscribe_clients_.emplace_back(
165+
new RedisAsioClient(io_service, context->subscribe_context()));
166+
}
61167
asio_async_auxiliary_client_.reset(
62168
new RedisAsioClient(io_service, primary_context_->async_context()));
63169
asio_subscribe_auxiliary_client_.reset(

src/ray/gcs/client.h

Lines changed: 16 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -24,21 +24,22 @@ class RAY_EXPORT AsyncGcsClient {
2424
/// Attach() must be called. To read and write from the GCS tables requires a
2525
/// further call to Connect() to the client table.
2626
///
27+
/// \param address The GCS IP address.
28+
/// \param port The GCS port.
29+
/// \param sharding If true, use sharded redis for the GCS.
2730
/// \param client_id The ID to assign to the client.
2831
/// \param command_type GCS command type. If CommandType::kChain, chain-replicated
2932
/// versions of the tables might be used, if available.
30-
AsyncGcsClient(const ClientID &client_id, CommandType command_type);
31-
AsyncGcsClient(const ClientID &client_id);
32-
AsyncGcsClient(CommandType command_type);
33-
AsyncGcsClient();
33+
AsyncGcsClient(const std::string &address, int port, const ClientID &client_id,
34+
CommandType command_type, bool is_test_client);
35+
AsyncGcsClient(const std::string &address, int port, const ClientID &client_id,
36+
bool is_test_client);
37+
AsyncGcsClient(const std::string &address, int port, CommandType command_type);
38+
AsyncGcsClient(const std::string &address, int port, CommandType command_type,
39+
bool is_test_client);
40+
AsyncGcsClient(const std::string &address, int port);
41+
AsyncGcsClient(const std::string &address, int port, bool is_test_client);
3442

35-
/// Connect to the GCS.
36-
///
37-
/// \param address The GCS IP address.
38-
/// \param port The GCS port.
39-
/// \param sharding If true, use sharded redis for the GCS.
40-
/// \return Status.
41-
Status Connect(const std::string &address, int port, bool sharding);
4243
/// Attach this client to a plasma event loop. Note that only
4344
/// one event loop should be attached at a time.
4445
Status Attach(plasma::EventLoop &event_loop);
@@ -71,7 +72,7 @@ class RAY_EXPORT AsyncGcsClient {
7172
Status GetExport(const std::string &driver_id, int64_t export_index,
7273
const GetExportCallback &done_callback);
7374

74-
std::shared_ptr<RedisContext> context() { return context_; }
75+
std::vector<std::shared_ptr<RedisContext>> shard_contexts() { return shard_contexts_; }
7576
std::shared_ptr<RedisContext> primary_context() { return primary_context_; }
7677

7778
private:
@@ -88,9 +89,9 @@ class RAY_EXPORT AsyncGcsClient {
8889
std::unique_ptr<ProfileTable> profile_table_;
8990
std::unique_ptr<ClientTable> client_table_;
9091
// The following contexts write to the data shard
91-
std::shared_ptr<RedisContext> context_;
92-
std::unique_ptr<RedisAsioClient> asio_async_client_;
93-
std::unique_ptr<RedisAsioClient> asio_subscribe_client_;
92+
std::vector<std::shared_ptr<RedisContext>> shard_contexts_;
93+
std::vector<std::unique_ptr<RedisAsioClient>> shard_asio_async_clients_;
94+
std::vector<std::unique_ptr<RedisAsioClient>> shard_asio_subscribe_clients_;
9495
// The following context writes everything to the primary shard
9596
std::shared_ptr<RedisContext> primary_context_;
9697
std::unique_ptr<DriverTable> driver_table_;

src/ray/gcs/client_test.cc

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -28,9 +28,8 @@ static inline void flushall_redis(void) {
2828
class TestGcs : public ::testing::Test {
2929
public:
3030
TestGcs(CommandType command_type) : num_callbacks_(0), command_type_(command_type) {
31-
client_ = std::make_shared<gcs::AsyncGcsClient>(command_type_);
32-
RAY_CHECK_OK(client_->Connect("127.0.0.1", 6379, /*sharding=*/false));
33-
31+
client_ = std::make_shared<gcs::AsyncGcsClient>("127.0.0.1", 6379, command_type_,
32+
/*is_test_client=*/true);
3433
job_id_ = JobID::from_random();
3534
}
3635

@@ -60,7 +59,10 @@ class TestGcsWithAe : public TestGcs {
6059
public:
6160
TestGcsWithAe(CommandType command_type) : TestGcs(command_type) {
6261
loop_ = aeCreateEventLoop(1024);
63-
RAY_CHECK_OK(client_->context()->AttachToEventLoop(loop_));
62+
RAY_CHECK_OK(client_->primary_context()->AttachToEventLoop(loop_));
63+
for (auto &context : client_->shard_contexts()) {
64+
RAY_CHECK_OK(context->AttachToEventLoop(loop_));
65+
}
6466
}
6567

6668
TestGcsWithAe() : TestGcsWithAe(CommandType::kRegular) {}

0 commit comments

Comments
 (0)