Global Control Store
GCS (Global Control Store) 是 Ray 的全局控制存储系统,它是 Ray 的核心组件之一,负责存储和管理集群的元数据信息,它确保了整个集群的正常运行和高效调度。从图中可以看出来GCS 的重要性,其核心的功能包括:
- 中心化管理: 提供全局视图、统一控制平面和元数据管理
- 调度支持: 负责资源分配、任务调度和负载均衡
- 容错机制: 实现状态恢复、故障检测和高可用性保证
- 性能优化: 包含缓存机制、异步操作和批量更新
- 监控和调试: 支持状态追踪、性能监控和故障诊断
与其他组件的交互
Client/Driver
│
▼
Raylet (Local Scheduler)
│
▼
GCS (Global Control Store)
│
▼
Redis/Other Storage Backend
代码概览
核心组件
GCS由以下几个核心组件组成:
- 节点管理 (GcsNodeManager)
void GcsServer::InitGcsNodeManager() {
gcs_node_manager_ = std::make_unique<GcsNodeManager>(
gcs_publisher_.get(),
gcs_table_storage_.get(),
raylet_client_pool_.get());
}
负责管理集群中的节点
- 负责管理集群中的节点
- 处理节点注册、心跳和故障检测
- Actor 管理 (GcsActorManager)
void GcsServer::InitGcsActorManager() {
gcs_actor_manager_ = std::make_unique<GcsActorManager>(
std::move(scheduler),
gcs_table_storage_.get(),
gcs_publisher_.get());
}
管理 Actor 的创建、销毁和重建
- 管理 Actor 的创建、销毁和重建
- 处理 Actor 调度和故障恢复
- 资源管理 (GcsResourceManager)
void GcsServer::InitGcsResourceManager() {
gcs_resource_manager_ = std::make_unique<GcsResourceManager>(
cluster_resource_scheduler_->GetClusterResourceManager(),
*gcs_node_manager_);
}
- 管理集群资源
- 追踪节点资源使用情况
- 任务管理 (GcsTaskManager)
void GcsServer::InitGcsTaskManager() {
gcs_task_manager_ = std::make_unique<GcsTaskManager>();
}
- 管理任务的调度和执行
存储实现
GCS 支持两种存储模式:
enum class StorageType {
IN_MEMORY, // 内存存储
REDIS_PERSIST, // Redis 持久化存储
UNKNOWN
};
核心功能
- 服务启动流程:
void GcsServer::Start() {
// 1. 初始化 KV 管理器
InitKVManager();
// 2. 异步加载 GCS 表数据
auto gcs_init_data = std::make_shared<GcsInitData>();
gcs_init_data->AsyncLoad();
// 3. 初始化各个管理器
InitGcsNodeManager();
InitGcsActorManager();
InitGcsResourceManager();
// ...
// 4. 启动 RPC 服务
rpc_server_.Run();
}
- 事件监听机制:
void GcsServer::InstallEventListeners() {
// 节点事件
gcs_node_manager_->AddNodeAddedListener();
gcs_node_manager_->AddNodeRemovedListener();
// Worker 事件
gcs_worker_manager_->AddWorkerDeadListener();
// Job 事件
gcs_job_manager_->AddJobFinishedListener();
}
- 调度机制
// 资源变化时触发调度
gcs_resource_manager_->AddResourcesChangedListener([this] {
// 调度待处理的 placement groups
gcs_placement_group_manager_->SchedulePendingPlacementGroups();
// 调度待处理的任务
cluster_task_manager_->ScheduleAndDispatchTasks();
});
以上就是 Ray GCS 的核心实现架构,它作为 Ray 的中央控制组件,协调管理整个集群的运行。如果只是想大概了解下功能,下面的内容可以跳过了。如果你想深入地了解代码细节,可以接着往下翻。
核心代码详解
GCS核心的代码在 [https://github.com/ray-project/ray/tree/master/src/ray/gcs](https://github.com/ray-project/ray/tree/master/src/ray/gcs)
里面,现在正式开始代码级分析:
gcs_client
accessor
- accessor访问器主要用于访问 GCS 中存储的不同类型的数据
- 主要的访问器类包括:
// 访问 Actor 相关信息
class ActorInfoAccessor
// 访问 Job 相关信息
class JobInfoAccessor
// 访问节点相关信息
class NodeInfoAccessor
// 访问节点资源信息
class NodeResourceInfoAccessor
// 访问错误信息
class ErrorInfoAccessor
// 访问任务信息
class TaskInfoAccessor
// 访问 Worker 信息
class WorkerInfoAccessor
// 访问 Placement Group 信息
class PlacementGroupInfoAccessor
// 访问内部 KV 存储
class InternalKVAccessor
-
每个访问器类都提供了一系列异步和同步方法来:
- 获取(Get)数据
- 添加(Add)数据
- 更新(Update)数据
- 删除(Remove)数据
- 订阅(Subscribe)数据变更
-
主要的设计特点:
- 大量使用异步操作,通过回调函数处理结果
virtual Status AsyncGet(const ActorID &actor_id, const OptionalItemCallback<rpc::ActorTableData> &callback);
- 支持超时机制
virtual Status AsyncGetByName( const std::string &name, const std::string &ray_namespace, const OptionalItemCallback<rpc::ActorTableData> &callback, int64_t timeout_ms =