TiKV 的 partitioned raft kv 特性

当前版本的 TiKV 中使用了单个 RocksDB 实例去储存全部的数据,甚至在更早期,Raft Log 和 payload 也存放在一起。

一些设计上的讨论

关于 Region 大小的讨论

较小的 Region 的好处:

  1. 每个 Region 中较低的并发
  2. 更加快速的调度

较大的 Region 的好处:

  1. Placement Driver 的压力变小
  2. CompactLog、Heartbeat 等网络开销变小
  3. 1PC 的事务更多

Raft Group 和 Data Range 的对应关系

TiKV 中,Raft Group 和 Region 严格一一对应。TiKV 中 Region 管理一段范围内的数据,在其他一些实现中,这段范围可能被称作 Shard、Partition 等。讨论下这个设计:

  1. Raft 本身和 Region 数据的版本引入了全序关系
    首先,Raft Admin Command 会穿插在写入之间形成很多 barrier,带来额外的持久化负担。
    然后,这导致了新创建的 peer 只能通过 Snapshot 追进度的情况。从 Raft 协议来看,ConfChange 之前的日志的提交和复制应当遵守 C_old 的配置项目,但是它并没有禁止进入 C_new 状态的 Leader 给新 peer 发送 ConfChange 之前的日志。但考虑到如果新 peer 还在处理 C_old 时代的日志,它的本地状态比如 RegionLocalState 肯定对应了 C_old,这个时候它接受到了一个“不认识”的 store 的 AppendEntries,这是比较奇怪的。
  2. Raft Group 不稳定
    Split 会分出独立的 Raft Group,给 pd 调度带来压力。也变相增大了 recover 的工作量。
    Merge 两个 Region 会销毁一个 Raft Group,这里面有不少 corner case。比如 Leader 关掉后的孤儿 Learner 问题。

我觉得可能 Spanner 的架构会更好一点。也就是说:

  1. 一个 “Spanner Region” 一个 Raft Group,但这个 “Spanner Region” 不再和某个 Key range 绑定。
  2. 一个 “Spanner Region” 下可以被调度多个 Key range。例如有局部性的 Key range 可以被调度在一起,或者处于打散负载的目的可以将 Key range 进行随机的分布。

Raft Group 和 Data Range 分开的架构

即使 Raft Group 和 Data Range 是一一对应的,那么在这之上还有一些设计:

  1. 全局需要维护多少个 Raft Group?
    一个 Raft Group 可能需要处理不同 Key range 的数据。但全局关系肯定是过强了,破坏了 Partitioning 的初衷。所以会更倾向于引入乱序 Apply 机制来提高 RSM 的吞吐量。
  2. 谁有权限写 Data Range?
    一般来说,会将对应的 Raft Group Leader 设置为 Data Range 的 “Leader”,让它来处理写入。这样做的好处是可以减少一次 RPC。

另外,分开的实现还有个好处,就是如果 Raft 层的 Leader 发生切换,Data Range 层的读取不会收到影响,而是可以 bypass 掉 Raft 层。CRDB 就是这样实现的,也就是类似是 Data Range 上的 LeaseRead。相比之下,TiKV 的 LeaseRead 和 Raft Leader 的生命周期是绑定的。

在本文的后面还会提到 Follower Read 相关的话题,特别是它和乱序 apply 的关系。我个人觉得,如果将 Data Range 和 Raft Group 分开,我们仍然是可以实现 Follower Read 的。如果你把 Data Range 看成一个 RSM,那这种架构就类似于一个 Raft Group 去管理多个 RSM。我们在 Data Range 上维护一个 index,应该就行了。

Partitioned RaftKV 相关

和 Mono-store RaftKV 的兼容性问题

新架构简化了 Snapshot 的生成和注入流程:

  1. 在生成时,只需要对当前 Region 对应的 RocksDB 做一个 Snapshot 就行。这个 Snapshot 包含的数据可以新于 Raft Local State。
  2. 在注入时,只需要重命名 RocksDB 文件夹即可。不需要处理 range overlap 的问题。因此不需要引入单线程的 region worker。

因此 Mono-store RaftKV 需要处理下列问题:

  1. RocksDB 数据和 Raft 状态不一致。
  2. Snapshot 的 Range 可能和其他本地 Region Overlap。

不光是 Snapshot,在 Partitioned RaftKV 中,Region Peer 之间也可能互相 Overlap。所幸这个场景只会出现在 BatchSplit 和调度 Peer 发生冲突的情况下。

在新架构中,Apply 的落盘也实现了异步化,现在下层引擎可以选择在任意时刻落盘数据,并且在落盘完毕后通知 raftstore。这对 TiFlash 来说是一件好事,我们可以由此来让 KVStore 的落盘不再阻塞。

采用更大的 Region 的性能影响

  1. 可以采用 Parallel Raft 的方式实现并行 Apply。
  2. 单个 Region 的 Apply 压力会增大,但是下层 RocksDB 的负担减轻了。相比于单个实例的 RocksDB,新架构的层数更少,并且并发写入也更少。后续还可以尝试支持多盘部署。

另一个考量点是如果集群中出现很多小表,那么大 Region 的效果不能完全展示:

  1. 因为编码的问题,table 编码不相邻的表不能被合并到同一个 Region 中。
  2. 相邻的 table 合并会给 TiFlash 带来不少问题。例如如果给一些小表添加 TiFlash 副本,并且这个小表被合并到一个大 Region 中,那么发来的 Snapshot 可能非常大,并且包含了大量 TiFlash 不需要的数据。此外,TiFlash 本身的存储引擎也需要做出调整。