导读
内存管理是数据库系统不可忽视的核心问题之一,它直接影响系统的性能、稳定性和成本效率。
良好的内存管理有助于提高资源利用率、降低硬件成本、提升系统可扩展性,从而保障流畅的用户体验; 反之, 如果采取「无为而治」的方式管理内存 ,最终将难逃操作系 统 O OM Killer 的审判 —— 进程被强制终止,服务暂时中断。 在高可用要求的环境下,即使是几秒钟的服务中断也可能造成经济损失。 因此,数据库系统必须具备完善的内存管理机制。
作为知名分布式数据库 TiDB 的基石,TiKV 是一款成熟的分布式存储引擎,对内存管理非常重视。本文作为 TiKV 源码解读系列的续篇,将展开探讨其内存管理机制。本文将重点聚焦于 Raft Store 模块的内存管理。
本文包含以下内容:
- 概览 TiKV 各模块的内存消耗,建立全局认知;
- 深入 Raft Store,介绍写请求的处理流程;
- 讨论处理流程中的内存消耗,并介绍 Raft Store 的内存监控与管理;
- 具体的源码实现。
上图展示了 TiKV 的主要模块及其内存消耗来源,解释如下:
- Storage :内存用于维护写事务的 Latch 以及 resolved-ts 模块追踪写事务的 Lock。
- Coprocessor :内存用于缓存查询中间结果,与 gRPC 响应客户端速度相关,响应缓慢时内存消耗堆积。
- Raft Store :内存用于 Raft Messages & Entries & Committed Entries、Entry Cache 和 Region metadata。下文会详细介绍这三大部分。
- Raft Engine :基于 Append-only Log 结构,可视为只有 L0 的 LSM-tree,内存用于 Memtable 和 Block Cache,后者默认使用 2% 的系统内存。
- KV Engine (RocksDB) :同样存在 LSM-tree 的内存消耗。Block Cache 用于缓存用户数据,默认分配 45% 的系统内存以保障读性能稳定。
在这些模块中,读请求主要在 Coprocessor 和 KV Engine 的 Block Cache 消耗内存,因为无需 Latch 和 Lock,并且可通过 Lease Read 避免走 Raft Store。相比之下,写请求才是 Raft Store 内存消耗的主要来源。因此,接下来我们将重点分析写请求在 Raft Store 的处理流程,以弄清 Raft Store 内存消耗的来源,为后面分析其内存管理机制打下基础。
在 Raft Store 中,写请求的处理流程相对复杂,不仅涉及 Leader 和 Follower 的不同处理逻辑,还需要经过 Raft 流程的多个阶段。为了提升性能,TiKV 引入了多种优化技术,如 Async 和 Pipeline,这使得流程更加复杂。
我曾尝试用一幅序列图来展示整个写入流程,但最终得到的图片内容要素过多,不便展示。因此我决定将写入流程拆分为三幅图展示。这三幅图分别对应 Leader 和 Follower 处理逻辑的三个阶段 —— PreRaft、Raft 和 Apply 。每个阶段的工作简单概括如下:
- PreRaft :负责接受,预处理和传递请求/消息给 Raft 系统处理;
- Raft :负责日志的复制与持久化;
- Apply :负责将日志应用 到状态机。
写请求的生命周期实际经历了六个步骤:Leader PreRaft -> Leader Raft -> Follower PreRaft -> Follower Raft -> Leader Apply -> Follower Apply。这六个步骤构成一个流水线,实现了时间并行(Temporal Parallelism)。
话不多说,接下来请看三幅图(三个阶段):
前排提醒: 序列图的参与者除明确指出的 Service、Module 和 Thread 外,均为 TiKV 代码中的 struct 实例对象(下同); 序列图刻意忽略了 Raft Batch 处理以简化流程(下同)。
首先是 PreRaft 阶段的流程,如上图所示,具体解释如下:
- Leader :gRPC Service 收到写请求后,通过 Storage 模块和 RaftRouter 转发,生成 Raft Command,并通过 channel 将其发送给目标 Region 的 PeerFsm,最终交由 RaftBatchSystem 处理。
- Leader 或 Followe r:收到 Raft Message 后,经过类似的流程后,将消息交由 RaftBatchSystem 处理。
特别提醒: 虚线箭头表示异步操作,既可以是直接的异步调用,也可以是由异步事件触发的操作。
其次是 Raft 阶段的流程,如上图所示,具体解释如下:
-
Leader :首先 Raft 线程从 RaftBatchSystem 拉取 PeerFsm 进行处理,向 Raft 模块发起 Propose,得到日志 Entries(未提交的 Entries,区别于下一幅图将介绍的 Committed Entries,)。随后将这些 Entries 异步地发往 Follower。接着将它们缓存到 Entry Cache(下文会详细介绍 Entry Cache),并发起异步地日志持久化操作。最后推进下 Raft 状态。
-
Follower :处理流程与 Leader 类似,但处理的是 Append 而非 Propose。Follower 在处理 Append