{
    "componentChunkName": "component---src-templates-blog-blog-detail-tsx",
    "path": "/blog/ticdc-source-code-reading-2",
    "result": {"pageContext":{"blog":{"id":"Blogs_445","title":"TiCDC 源码阅读（二）TiKV CDC 模块介绍","tags":["TiCDC"],"category":{"name":"产品技术解读"},"summary":"本文是 TiCDC 源码解读的第二篇，将介绍 TiCDC 的重要组成部分，TiKV 中的 CDC 模块。","body":"## 内容概要\n\nTiCDC 是一款 TiDB 增量数据同步工具，通过拉取上游 TiKV 的数据变更日志，TiCDC 可以将数据解析为有序的行级变更数据输出到下游。\n\n本文是 TiCDC 源码解读的第二篇，将于大家介绍 TiCDC 的重要组成部分，TiKV 中的 CDC 模块。我们会围绕 4 个问题和 2 个目标展开。\n\n1. TiKV 中的 CDC 模块是什么？\n2. TiKV 如何输出数据变更事件流？\n3. 数据变更事件有哪些？\n4. 如何确保完整地捕捉分布式事务的数据变更事件？\n\n希望在回答完这4个问题之后，大家能：\n\n- 🔔 了解数据从 TiDB 写入到 TiKV CDC 模块输出的流程。\n- 🗝️ 了解如何完整地捕捉分布式事务的数据变更事件。\n\n在下面的内容中，我们在和这两个目标相关的地方会标记上 🔔 和 🗝️，以便提醒读者留意自己感兴趣的地方。\n\n## TiKV 中的 CDC 模块是什么？\n\n### CDC 模块的形态\n\n从代码上看，CDC 模块是 TiKV 源码的一部分，它是用 rust 写的，在 TiKV 代码库里面；从运行时上看，CDC 模块运行在 TiKV 进程中，是一个线程，专门处理 TiCDC 的请求和数据变更的捕捉。\n\n### CDC 模块的作用\n\nCDC 模块的作用有两个：\n\n1. 它负责捕捉实时写入和读取历史数据变更。这里提一下历史数据变更指已经写到 rocksdb 里面的变更。\n2. 它还负责计算 resolved ts。这个 resolved ts 是 CDC 模块里面特有的概念，形式上是一个 uint64 的 timestamp。它是 TiKV 事务变更流中的 perfect watermark，perfect watermark 的详细概念参考《Streaming System》的第三章，我们可以用 resolved ts 来告知下游，也就是 TiCDC，在 tikv 上所有 commit ts 小于 resolved ts 事务都已经完整发送了，下游 TiCDC 可以完整地处理这批事务了。\n\n### CDC 模块的代码分布\n\nCDC 模块的代码在 TiKV 代码仓库的 `compoenetns/cdc` 和 `components/resolved_ts` 模块。我们在下图中的黑框里面用红色标注了几个重点文件。\n\n在 `delegate.rs` 文件中有个同名的 `Delegate` 结构体，它可以认为是 Region 在 CDC 模块中的“委派”，负责处理这个 region 的变更数据，包括实时的 raft 写入和历史增量数据。\n\n在 `endpoint.rs` 文件中有个 `Endpoint` 结构体，它运行在 CDC 的主线程中，驱动整个 CDC 模块，上面的 delegate 也是运行在整个线程中的。\n\n`initializer.rs` 文件中的 `Initializer` 结构体负责增量扫逻辑，同时也负责 delegate 的初始化，这里的增量扫就是读取保存在 rocksdb 中的历史数据变更。\n\n`service.rs` 文件中的 `Service` 结构体，它实现了 ChagneData gRPC 服务，运行在 gRPC 的线程中，它负责 TiKV 和 TiCDC 的 RPC 交互，同时它和 `Endpoint` 中的 `Delegate` 和 `Initializer` 也会有交互，主要是接受来自它俩的数据变更事件，然后把这些事件通过 RPC 发送给 TiCDC。\n\n最后一个重要文件是 `resolver.rs`，它与上面的文件不太一样，在 resolve_ts 这个 component 中，里面的 `Resolver` 负责计算 resolved ts。\n\n![1.PNG](https://img1.www.pingcap.com/prod/1_e91f78042c.PNG)\n\n## TiKV 如何输出数据变更事件流？\n\n我们从端到端的角度完整地走一遍数据的写入和流出。下图概括了数据的流动，我们以数据保存到磁盘为界，红色箭头代表数据从 TiDB 写入 TiKV 磁盘的方向，蓝色箭头代表数据从 TiKV 磁盘流出到 TiCDC 的方向。\n\n![UML 图.jpg](https://img1.www.pingcap.com/prod/UML_bb75620add.jpg)\n\n### TiDB -> TiKV Service\n \n- txn prewrite: [Tikv::kv_prewrite(PrewriteRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/server/service/kv.rs#L242)\n- txn commit: [Tikv::kv_commit(CommitRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/server/service/kv.rs#L263)\n\n我们看下从 TiDB 指向 TiKV 的红线。我们知道数据来自 TiDB 的事务写入，对于一个正常的事务来说，TiDB 需要分两次调用 TiKV 的 gRPC 接口，分别是 kv_prewrite 和 kv_commit，对应了事务中的 prewrite 和 commit，在 request 请求中包含了要写入或者删除的 key 和它的 value，以及一些事务的元数据，比如 start ts，commit ts 等。\n\n### TiKV Service -> Txn\n\n- txn prewrite: [Storage::sched_prewrite(PrewriteRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/server/service/kv.rs#L2189-L2241)\n- txn commit: [Storage::sched_commit(CommitRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/server/service/kv.rs#L2271-L2283)\n\n我们再看从 gRPC 指向 Txn 的红线。它代表 RPC 请求从 gRPC 模块流到事务模块的这一步。这里相应的也有两个 API 的调用，分别是 `sched_prewrite` 和 `sched_commit`，在这两个 API 中，事务模块会对 request 做一些检查，比如检查 write conflict，计算 commit ts 等（事务的细节可以参考 TiKV 的源码阅读文章，在这里就先跳过了。）\n\n### Txn -> Raftstore\n\n- txn prewrite: [Engine::async_write_ext(RaftCmdRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/storage/txn/scheduler.rs#L1323)\n- txn commit: [Engine::async_write_ext(RaftCmdRequest)](https://github.com/tikv/tikv/blob/v6.4.0/src/storage/txn/scheduler.rs#L1323)\n\n事务模块到 Raftstore 的红线代表：Request 通过检查后，会被事务模块序列化成对 KV 的操作，然后被组装成 `RaftCmdRequest`。`RaftCmdRequest` 再经由 `Engine::async_commit_ext API` 被发送至 Raftstore 模块。\n\n大家可以看到 prewrite 和 commit 都是变成了 `RaftCmdRequest`，也都是通过 `Engine::async_commit_ext` 发送到 Raftstore 模块的。这说明了什么呢？它说明了到 Engine 这一层，TiDB 的请求中的事务信息已经被“抹去”了，所有的事务信息都存到了 key 和 value 里面。\n\nRaftstore 模块会将这些 key value 提交到 Raft Log 中，如果 Raft Log Commit 成功，Apply 线程会将这些 key 和 value 写入到 Rocksdb。（这里面的细节可以参考 [TiKV 的源码阅读文章](https://cn.pingcap.com/blog/?tag=TiKV%20源码阅读)，在这里就先跳过了。）\n\n### Rafstore -> CDC\n\n- RaftCmd: [CoprocessorHost::on_flush_applied_cmd_batch(Vec<RaftCmdRequest>)](https://github.com/tikv/tikv/blob/v6.4.0/components/raftstore/src/store/fsm/apply.rs#L597)\n- Txn Record: [Engine::async_snapshot()](https://github.com/tikv/tikv/blob/v6.4.0/src/server/raftkv.rs#L431)\n  \n从这里起，数据开始流出了，从 Raftstore 到 CDC 模块有两条蓝线，对应这里的两个重要的 API，分别为 `on_flush_applied_cmd_batch` 实时数据的流出，和 `async_snapshot` 历史增量数据的流出（后面会说细节）。\n  \n### CDC -> gRPC -> TiCDC\n  \n- ChangeDataEvent: [Service::event_feed() -> ChangeDataEvent](https://github.com/tikv/tikv/blob/v6.4.0/components/cdc/src/service.rs#LL201C8-L201C18)\n  \n最后就是从 CDC 模块到 TiCDC 这几条蓝线了。数据进入 CDC 模块后，经过一系列转换，组装成 Protobuf message，最后交给 gRPC 通过 ChangeData service 中的 `EventFeed` 这个 RPC 发送到下游的 TiCDC。\n  \n### CDC 模块中的数据流动\n\n![UML 图 (1).jpg](https://img1.www.pingcap.com/prod/UML_1_f57939f8fc.jpg)\n\n上图示意了数据从 Raftstore 发送到 TiCDC 模块的细节。\n  \n数据从 Raftstore 到 CDC 模块，可以分成两个阶段，对应两条链路：\n  \n- **阶段 1，增量扫**，Initializer -> Delegate。\n  \n  Initializer 从 Raftstore 拿一个 Snapshot，然后在 Snapshot 上读一些历史数据变更，读的范围有两个维度：\n  \n    1. 时间维度 `(checkpoint ts, current ts]`，checkpoint ts 可以理解成 changefeed 上的 checkpoint，current ts 代表 PD leader 上的当前时间。\n  \n    2. key 范围 `[start key, end key)`，一般为 region 的 start key 和 end key。\n  \n- **阶段 2，实时写入监听**，CdcObserver -> Delegate\n  \n  `CdcObserver` 实现对实时写入的监听。它运行在 Raftstore 的 Apply 线程中，只有在 TiCDC 对一个 Region 发起监听后才会启动运行。我们知道所有的数据都是通过 Apply 线程写入的，所以说 `CdcObserver` 能轻松地在第一时间把数据捕捉到，然后交给 `Delegate` 。\n  \n我们再看一下数据从 CDC 模块到 gRPC 的流程，大体也有两部分。第一部分是汇总增量扫和实时写入；第二部分将这些数据是从 KV 数据反序列化成包含事务信息的 Protobuf message。我们再将这些事务结构体里面的信息给提取出来，填到一个 Protobuf message 里面。\n  \n### Raftstore 和 TiCDC 的交互\n\n![UML 图 (2).jpg](https://img1.www.pingcap.com/prod/UML_2_993f60ef5a.jpg)\n\n上图是 Raftstore 和 CDC 模块的交互时序图。第一条线是 TiCDC，第二条是 CDC 线程，第三条是 Raftsotre 线程，第四条是 Apply 线程，图中每个点都是发生在线程上的一些事件，包含发消息、收消息和进程内部的处理逻辑。在这里我们重点说 Apply 线程。\n  \nApply 线程在处理 Change 这个消息的时候，它会先要把缓存在内存中的 KV 的写入给刷到 RocksDB，然后获取 RocksDB 的 Snapshot，把 Snapshot 发送给 CDC 线程。这三步是串行的，保证了 Snapshot 可以看到之前所有的写入。有了这个机制保证，我们就可以确保 CDC 模块既不漏数据，也不多数据。\n  \n## 数据变更事件有哪些？\n\n![image.png](https://img1.www.pingcap.com/prod/image_8a167cddf7.png)\n\n数据变更事件可分为两大类，第一类是 Event；第二类是 ResolvedTs。上图是 CDC Protobuf 的简化版定义，只保留了关键的 field。我们从上到下看下这个 Protobuf 定义。\n  \n`EventFeed` 定义了 TiCDC 和 TiKV 之间的消息交互方式，TiCDC 在一个 RPC 上可以发起对多个 Region 的监听，TiKV 以 `ChangeDataEvent` 形式将多个 Region 的数据变更事件发送给 TiCDC。\n  \n`Event` 代表着是 Region 级别的数据变更事件，包含了至少一个用户数据变更事件或者或者 Region 元数据的变更。它们是从单条 Raft Log 翻译得到的。我们可以注意到 `Event` 被 `repeat` 修饰了，也就是它可能包含了一个 region 多个数据变更，也可能包含多个不同 region 的数据变更。\n  \n`Entries` 包含了多个 `Row`。因为在 `oneof` 里面不能出现 `repeated` ，所以我们用 `Entries` 包装了下。\n\n![image (1).png](https://img1.www.pingcap.com/prod/image_1_f3177e681a.png)\n\n`Row` 里面的内容非常接近 TiDB 层面的数据了，它是行级别的数据变更，包含：\n1. 事务的 start ts；\n2. 事务的 commit ts；\n3. 事务写入的类型，Prewrite/Commit/Rollback；\n4. 事务对数据的操作，`op_type` ，put 覆盖写一行和 delete 删除一行；\n5. 事务写入的 key；\n6. 事务写入的 value；\n7. 该事务之前的 value，old value 在很多 CDC 协议上都会有体现，比如说 MySQL 的 maxwell 协议中的 “old” 字段。\n  \n## 如何确保完整地捕捉分布式事务的数据变更事件？\n  \n### 什么是“完整”？\n  \n我们需要定义完整是什么。在这里，“完整”的主体是 TiDB 中的事务，我们知道 TiDB 的事务会有两个写入事件，第一个是 prewrite，第二是 commit 或者 rollback。同时，TiDB 事务可能会涉及多个 key，这些有可能分布在不同的 region 上。所以，我们说“完整”地捕捉一个事务需要捕捉它涉及的**所有的 key** 和**所有的写入事件**。\n\n![UML 图 (3).jpg](https://img1.www.pingcap.com/prod/UML_3_42aee1488c.jpg)\n\n上图描绘了一个涉及了三个 key 的事务，P 代表事务的 prewrite，C 代表事务的 commit，虚线代表一次捕捉。\n  \n前面两条虚线是不“完整”的捕捉，第一条虚线漏了所有 key 的 commit 事件，第二条虚线捕捉到了 k1 和 k2 的 prewrite 和 commit，但漏了 k3 的 commit。如果我们强行认为第二条虚线是“完整”的，则会破坏事务的原子性。\n  \n最后一条虚线才是“完整”的捕捉，因为它捕捉到了所有 key 的所有写入。\n  \n### 如何确认已经“完整”？\n  \n确认“完整”的方法有很多种，最简单的办法就是--等。一般来说，只要我们等的时间足够长，比如等一轮 GC lifetime，我们也能确认完整。但是这个办法会导致 TiCDC 的 RPO 不达标。\n  \n![UML 图 (4).jpg](https://img1.www.pingcap.com/prod/UML_4_2ed55abe00.jpg)\n\n上图最后两条虚线是两次“完整”的捕捉，假如第四条线十年之后才产生的，显然它对我们来说是没有意义的。第四条虽然是“完整”的，但是不是我们想要的。所以我们需要一种机制能够尽快地告知我们已经捕捉完整了，也就是图中第三条虚线，在时间上要尽可能地靠近最后一个变更的捕捉。那这个机制的话就是前面提到的 resolved ts。\n  \n### ResolvedTs 事件及性质\n\n![image (2).png](https://img1.www.pingcap.com/prod/image_2_370f9e1f45.png)\n\nResolvedTs 在 Protobuf 中的定义比较简单，一个 Region ID 数组和一个 resolved ts。它记录了**一批** Region 中**最小的** resolved ts，会混在数据变更事件流中发送给 TiCDC。从 resolved ts 事件生成的时候开始，TiDB 集群就不会产生 commit ts 小于 resolved ts 的事务了。从而 TiCDC 收到这个事件之后，便能确认这些 Region 上的数据变更事件的完整性了。\n  \n### resolved ts 的计算\n\n![image (3).png](https://img1.www.pingcap.com/prod/image_3_abee7d0464.png)\n\nResolved ts 的计算逻辑在 resolver.rs 文件中，可以用简单三行伪代码表示：\n  \n- 第一行，它要从 PD 那边取一个 TS，称它为 `min_ts`。\n- 第二行，我们拿 `min_ts` 和 Region 中的所有 lock 的 start ts 做比较，取最小值，我们称它为 `new_resolved_ts` 。\n- 我们拿 `new_resolved_ts` 和之前的 `resolved_ts` 做比较，取最大值，这就是当前时刻的 resolved ts。因为它小于所有 lock 的 start ts，所有它一定小于这些 lock 的未来的 commit ts。同时，在没有 lcok 的时候，`min_ts` 会变成 resolved ts，也是就当前时刻 PD 上最新的 ts 将会变成 resolved ts，这确保了它有足够的实时性。\n  \n### 数据变更事件流的例子\n\n![UML 图 (5).jpg](https://img1.www.pingcap.com/prod/UML_5_d5b69856c0.jpg)\n\n上图是一个数据变更事件流的例子，也就是 gRPC EventFeed 中的 `stream ChangeDataEvent`。\n  \n例子中有三个事务和三个 resolved ts 事件：\n  \n- 第一个事务涉及了 k1 和 k2，它的 start ts 是 1， commit ts 是2。\n- 第二个事务只包含了 k1 这一个 key，它的 start ts 是 3，commit ts 是 6，注意，这个事务在事件流中出现了乱序，它的 commit 先于 prewrite 出现在这条流中。\n- 第三个事务包含了 k2 的一个事务，注意它只有一个 prewrite 事件，commit 事件还没发生，是一个正在进行中的一个事务。\n- 第一个 resolved ts 事件中的 resolved ts 是 2，代表 commit ts 小于等于 2 的事务已经完整发送，在这个例子中可以把第一个事务安全的还原出来。\n- 第二个 resolved ts 事件中的 resolved ts 是 4，这时 k1 的 commit 事件已经发送了，但是 prewrite 事件没有，4 就阻止了还原第二个事务。\n- 第三个 resolved ts 事件出现后，我们就可以还原第二个事务了。\n  \n## 结尾\n  \n以上就是本文的全部内容。希望在阅读上面的内容后，读者能知道文章开头的四个问题和了解：\n  \n- 🔔数据从 TiDB 写入到 TiKV CDC 模块输出的流程\n- 🗝️了解如何完整地捕捉分布式事务的数据变更事件","date":"2022-12-22","author":"沈泰宁","fillInMethod":"writeDirectly","customUrl":"ticdc-source-code-reading-2","file":null,"relatedBlogs":[{"relatedBlog":{"body":"这一次 TiCDC 阅读系列文章将会从源码层面来讲解 TiCDC 的基本原理，希望能够帮助读者深入地了解 TiCDC 。本篇文章是这一系列文章的第一期，主要叙述了 TiCDC 的目的、架构和数据同步链路，旨在让读者能够初步了解 TiCDC，为阅读其他源码阅读文章起到一个引子的作用。\n\n## TiCDC 是什么？\n\nTiCDC 是 TiDB 生态中的一个数据同步工具，它能够将上游 TiDB集群中产生的增量数据实时的同步到下游目的地。除了可以将 TiDB 的数据同步至 MySQL 兼容的数据库之外，还提供了同步至 Kafka 和 s3 的能力，支持 canal 和 avro 等多种开放消息协议供其他系统订阅数据变更。\n\n![1.PNG](https://img1.www.pingcap.com/prod/1_3684c70859.PNG)\n\n上图描述了 TiCDC 在整个  TiDB 生态系统中的位置，它处于一个上游 TiDB 集群和下游其它数据系统的中间，充当了一个数据传输管道的角色。\n\n**TiCDC 典型的应用场景为搭建多套 TiDB 集群间的主从复制，或者配合其他异构的系统搭建数据集成服务。以下将从这两方面为大家介绍：**\n\n### 主从复制\n\n使用 TiCDC 来搭建主从复制的 TiDB 集群时，根据从集群的使用目的，可能会对主从集群的数据一致性有不同的要求。目前 TiCDC 提供了如下两种级别的数据一致性:\n\n![2.PNG](https://img1.www.pingcap.com/prod/2_70eda26c9e.PNG)\n\n- 快照一致性：通过开启 Syncpoint 功能，能够在实时的同步过程中，保证上下游集群在某个 TSO 的具备快照一致性。详细内容可以参考文档：[TiDB 主从集群的数据校验](https://docs.pingcap.com/zh/tidb/dev/upstream-downstream-diff)\n- 最终一致性：通过开启 Redo Log 功能，能够在上游集群发生故障的时候，保证下游集群的数据达到最终一致的状态。详细内容可以参考文档：[使用 Redo Log 确保数据一致性](https://docs.pingcap.com/zh/tidb/dev/replicate-between-primary-and-secondary-clusters#%E7%AC%AC-5-%E6%AD%A5%E4%BD%BF%E7%94%A8-redo-log-%E7%A1%AE%E4%BF%9D%E6%95%B0%E6%8D%AE%E4%B8%80%E8%87%B4%E6%80%A7)\n\n### 数据集成\n\n![3.PNG](https://img1.www.pingcap.com/prod/3_1cc9144622.PNG)\n\n目前 TiCDC 提供将变更数据同步至 Kafka 和 S3 的能力，用户可以使用该功能将 TiDB 的数据集成进其他数据处理系统。在这种应用场景下，用户对数据采集的实时性和支持的消息格式的多样性会由较高的要求。当前我们提供了多种可供订阅的消息格式(可以参考 [配置 Kafka](https://docs.pingcap.com/zh/tidb/dev/ticdc-sink-to-kafka#sink-uri-%E9%85%8D%E7%BD%AE-kafka))，并在最近一段时间内对该场景的同步速度做了一系列优化，读者可以从之后的文章中了解相关内容。\n\n## TiCDC 的架构\n\n![4.PNG](https://img1.www.pingcap.com/prod/4_67f2e4ea50.PNG)\n\n确保数据传输的稳定性、实时性和一致性是 TiCDC 设计的核心目标。为了实现该目标，TiCDC 采用了分布式架构和无状态的服务模式，具备高可用和水平扩展的特性。想要深入了解 CDC 的架构，我们需要先认识下面这些概念：\n\n### 系统组件\n\n- TiKV：\n  - TiKV 内部的 CDC 组件会扫描和拼装 kv change log。\n  - 提供输出 kv change logs 的接口供 TiCDC 订阅。\n- Capture：\n  - TiCDC 运行进程，多个 capture 组成一个 TiCDC 集群。\n  - 同步任务将会按照一定的调度规则被划分给一个或者多个 Capture 处理。\n\n### 逻辑概念\n\n- KV change log：TiKV 提供的隐藏大部分内部实现细节的的 row changed event，TiCDC 从 TiKV 拉取这些 Event。\n- Owner：一种 Capture 的角色，每个 TiCDC 集群同一时刻最多只存在一个 Capture 具有 Owner 身份，它负责响应用户的请求、调度集群和同步 DDL 等任务。\n- ChangeFeed：由用户启动同步任务，一个同步任务中可能包含多张表，这些表会被 Owner 划分为多个子任务分配到不同的 Capture 进行处理。\n- Processor：Capture 内部的逻辑线程，一个 Capture 节点中可以运行多个 Processor。每个 Processor 负责处理 ChangeFeed 的一个子任务。\n- TablePipeline：Processor 内部的数据同步管道，每个 TablePipeline 负责处理一张表，表的数据会在这个管道中处理和流转，最后被发送到下游。\n\n### 基本特性\n\n- 分布式：具备高可用能力，支持水平扩展。\n- 实时性：常规场景下提供秒级的同步能力。\n- 有序性：输出的数据行级别有序，并且提供 At least once 输出的保证。\n- 原子性：提供单表事务的原子性。\n\n## TiCDC 的生命周期\n\n认识了以上的基本概念之后，我们可以继续了解一下 TiCDC 的生命周期。\n\n### Owner\n\n![5.png](https://img1.www.pingcap.com/prod/5_352c474d49.png)\n\n首先，我们需要知道，TiCDC 集群的元数据都会被存储到 PD 内置的 Etcd 中。当一个 TiCDC 集群被部署起来时，每个 Capture 都会向 Etcd 注册自己的信息，这样 Capture 就能够发现彼此的存在。接着，各个 Capture 之间会竞选出一个 Owner ，Owner 选举流程在 [cdc/capture.go](https://github.com/pingcap/tiflow/blob/master/cdc/capture/capture.go#L393) 文件的 `campaignOwner` 函数内，下面的代码删除了一些错误处理逻辑和参数设置，只保留主要的流程：\n\n```go\nfor {\n        // Campaign to be the owner, it blocks until it been elected.\n        err := c.campaign(ctx)\n        ...\n        owner := c.newOwner(c.upstreamManager)\n        c.setOwner(owner)\n        ...\n        err = c.runEtcdWorker(ownerCtx, owner,...)\n        c.owner.AsyncStop()\n        c.setOwner(nil)\n}\n```\n\n每一个 Capture 进程都会调用该函数，进入一个竞选的 Loop 中，每个  Capture 都会持续不断地在竞选 Owner。同一时间段内只有一个 Capture 会当选，其它候选者则会阻塞在这个 Loop 中，直到上一个 Owner 退出就会有新的 Capture 当选。\n\n最后真正的竞选是通过在 `c.campaign(ctx)` 函数内部调用 Etcd 的 `election.Campaign` 接口实现的，Etcd 保证了同一时间只有一个 Key 能够当选为 Owner。由于 Etcd 是高可用的服务，TiCDC 借助其力量实现了天然的高可用。\n\n竞选到 Owner 角色的 Capture 会作为集群的管理者，也负责监听和响应来自用户的请求。\n\n### ChangeFeed\n\nTiCDC 集群启动完毕之后，用户即可使用 TiCDC 命令行工具或者 OpenAPI 创建 ChangeFeed (同步任务)。\n一个 ChangeFeed 被创建之后，Owner 会负责对它进行检查和初始化，然后将以表为单位将划分为多个子任务分配给集群内的 Capture 进行同步。同步任务初始化的代码在 [cdc/owner/changefeed.go](https://github.com/pingcap/tiflow/blob/master/cdc/owner/changefeed.go#L404) 文件中。该函数的主要工作为：\n\n1. 向上游查询该同步任务需要同步的表的 Schema 信息，为接下来调度器分配同步任务做准备。\n2. 创建一个 `ddlPuller` 来拉取 DDL 。因为我们需要在同步的过程中保持多个 Capture 节点上 Schema 信息的一致，并且保证 DML 与 DDL 同步顺序。所以我们选择仅由 Owner 这个拥有 ChangeFeed 所以信息的角色同步 DDL。\n3. 创建 `scheduler` ，它会负责把该同步任务拆分成多个子任务，发送给别的 Capture 进行处理。\n\nCapture 接收到 Owner 发送过来的子任务之后，就会创建出一个 Processor 来处理它接收到的子任务，Processor 会为每张表创建出一个 TablePipeline 来同步对应的表的数据。Processor 会周期性的把每个 TablePipeline 的状态和进度信息汇报给 Owner，由 Owner 来决定是否进行调度和状态更新等操作。\n总而言之，TiCDC 集群和同步任务的状态信息会在 Owner 和 Processor 之间流转，而用户需要同步的数据信息则通过 TablePipeline 这个管道传递到下游，下一个小节将会对 TablePipeline 进行讲解，理解了它，就能够理解 TiCDC 是怎么同步数据的。\n\n### TablePipeline\n\n![6.png](https://img1.www.pingcap.com/prod/6_156ee3be14.png)\n\n顾名思义，TablePipeline 是一个表数据流动和处理的管道。Processor 接收到一个同步子任务之后，会为每一张表创建出一个 TablePipeline，如上图所示，它主要由 Puller、Sorter、Mounter 和 Sink 构成。\n\n- Puller： 负责拉取对应表在上游的变更数据，它隐藏了内部大量的实现细节，包括与 TiKV CDC 模块建立 gRPC 连接和反解码数据流等。\n- Sorter： 负责对 Puller 输出的乱序数据进行排序，并且会把 Sink 来不及消费的数据进行落盘，起到一个蓄水池的作用。\n- Mounter：根据事务提交时的表结构信息解析和填充行变更，将行变更转化为 TiCDC 能直接处理的数据结构。在这里，Mounter 需要和一个叫做 SchemaStorage 的组件进行交互，这个组件在 TiCDC 内部维护了所需表的 Schema 信息，后续会有内容对这其进行讲解。\n- Sink：将 Mounter 处理过后的数据进行编解码，转化为 SQL 语句或者 Kafka 消息发送到对应下游。\n\n这种模块化的设计方式，比较有利于代码的维护和重构。值得一提的是，如果你对 TiCDC 有兴趣，希望能够让它接入到当前 CDC 还不支持的下游系统，那么只要自己编码实现一个对应的 Sink 接口，就可以达到目的。\n接下来，我们以一个具体例子的方式来讲解数据在 TiCDC 内部的流转。假设我们现在建立如下表结构：\n\n```sql\nCREATE TABLE TEST(\n   NAME VARCHAR (20)     NOT NULL,\n   AGE  INT              NOT NULL,\n   PRIMARY KEY (NAME)\n);\n\n+-------+-------------+------+------+---------+-------+\n| Field | Type        | Null | Key  | Default | Extra |\n+-------+-------------+------+------+---------+-------+\n| NAME  | varchar(20) | NO   | PRI  | NULL    |       |\n| AGE   | int(11)     | NO   |      | NULL    |       |\n+-------+-------------+------+------+---------+-------+\n```\n\n此时，在上游 TiDB 执行以下 DML：\n\n```sql\nINSERT INTO TEST (NAME,AGE)\nVALUES ('Jack',20);\n\nUPDATE TEST\nSET AGE = 25\nWHERE NAME = 'Jack';\n```\n\n下面我们就来看一看这两条 DML 会通过什么样的形式经过 TablePipeline ，最后写入下游。\n\n#### Puller 拉取数据\n上文中提到 Puller 负责与 TiKV CDC 组件建立 gPRC 连接然后拉取数据，这是 [/pipeline/puller.go](https://github.com/pingcap/tiflow/blob/master/cdc/processor/pipeline/puller.go#L67) 中的 Puller 大致的工作逻辑：\n\n```go\nn.plr = puller.New(... n.startTs, n.tableSpan(),n.tableID,n.tableName ...)\nn.wg.Go(func() error {\n   ctx.Throw(errors.Trace(n.plr.Run(ctxC)))\n   ...\n})\nn.wg.Go(func() error {\n   for {\n      select {\n      case <-ctxC.Done():\n         return nil\n      case rawKV := <-n.plr.Output():\n         if rawKV == nil {\n            continue\n         }\n         pEvent := model.NewPolymorphicEvent(rawKV)\n         sorter.handleRawEvent(ctx, pEvent)\n      }\n   }\n})\n```\n\n以上是经过简化的代码，可以看到在 `puller.New` 方法中，有两个比较重要的参数 `startTs` 和 `tableSpan()`，它们分别从时间和空间这两个维度上描述了我们想要拉取的数据范围。在 Puller 被创建出来之后，下面部分的代码分别启动了两个 goroutine，第一个负责运行 Puller 的内部逻辑，第二个则是等待 Puller 输出数据，然后把数据发给 Sorter。从 `plr.Output()` 中吐出来的数据长这个样子：\n\n```go\n// RawKVEntry notify the KV operator\ntype RawKVEntry struct {\n   OpType OpType `msg:\"op_type\"`\n   Key    []byte `msg:\"key\"`\n   // nil for delete type\n   Value []byte `msg:\"value\"`\n   // nil for insert type\n   OldValue []byte `msg:\"old_value\"`\n   StartTs  uint64 `msg:\"start_ts\"`\n   // Commit or resolved TS\n   CRTs uint64 `msg:\"crts\"`\n   ...\n}\n```\n\n所以，在上游 TiDB 写入的那两条 DML 语句，在到达 Puller 的时候会是这样这样的一个数据结构\n\n![7.png](https://img1.www.pingcap.com/prod/7_39e4d6bcc1.png)\n\n我们可以看到 Insert 语句扫描出的数据只有 value 没有 old_value，而 Update 语句则被转化为一条既有 value 又有 old_value 的行变更数据。\n\n这样这两条数据就成功的被 Puller 拉取到了 TiCDC，但是因为 TiDB 中一张表的数据会被分散到多个 Region 上，所以 Puller 会与多个 TiKV Region Leader 节点建立连接，然后拉取数据。那实际上 TiCDC 拉取到的变更数据可能是乱序的，我们需要对拉取到的所有数据进行排序才能正确的将事务按照顺序同步到下游。\n\n#### Sorter 排序\n\nTablePipeline 中的 Sorter 只是一个拥有 Sorter 名字的中转站，实际上负责对数据进行排序的是它背后的 Sorter Engine，Sorter Engine 的生命周期是和 Capture 一致的，一个 Capture 节点上的所有 Processor 会共享一个 Sorter Engine。想要了解它是怎么工作的，可以阅读 [EventSorter 接口](https://github.com/pingcap/tiflow/blob/master/cdc/sorter/sorter.go#L32)和其具体实现的相关代码。\n\n在这里，我们只需要知道数据进入 TablePipeline 中的 Sorter 后会被排序即可。假设我们现在除了上述的两条数据之外，在该表上又进行了其他的写入操作，并且该操作的数据在另外一个 Region。最终 Puller 拉到的数据如下：\n\n![8.png](https://img1.www.pingcap.com/prod/8_6fb8394f77.png)\n\n除了数据之外，我们还可以看到 `Resolved` 的事件，这是一个在 TiCDC 系统中很重要的时间标志。当 TiCDC 收到 `Resolved` 时，**可以认为小于等于这个时间点提交的数据都已经被接收了，并且以后不会再有早于这个时间点的数据再发送下来，此时 TiCDC 可以此为界限来将收到的数据同步至下游。**\n\n此外，我们可以看到拉取到的数据并不是按照 commit_ts 严格排序的，Sorter 会根据 commit_ts 将它们进行排序，最终得到如下的数据：\n\n![9.png](https://img1.www.pingcap.com/prod/9_10f6237293.png)\n\n现在排好顺序的事件就可以往下游同步了，但是在这之前我们需要先对数据做一些转换，因为此时的数据是从 TiKV 中扫描出的 key-value，它们实际上只是一堆 bytes 数据，而不是下游想要消费的消息格式。\n\n#### Mounter 解析\n\n以上的 Event 数据从 Sorter 出来之后，Mounter 会根据其对应的表的 Schema 信息将它还原成按照表结构组织的数据。\n\n```go\ntype RowChangedEvent struct {\n   StartTs  uint64\n   CommitTs uint64\n   Table    *TableName\n   ColInfos []rowcodec.ColInfo\n   Columns      []*Column\n   PreColumns   []*Column\n   IndexColumns [][]int\n   ...\n}\n```\n\n可以看到，该结构体中还原出了所有的表和列信息，并且 Columns 和 PreColumns 就对应于 value 和 old_value。当 TiCDC 拿到这些信息之后我们就可以将数据继续下发至 Sink 组件，让其根据表信息和行变更数据去写下游数据库或者生产 Kafka 消息。值得注意的是，Mounter 进行的是一项 CPU 密集型工作，当一个表中所包含的字段较多时，Mounter 会消耗大量的计算资源。\n\n#### Sink 下发数据\n\n当 `RowChangedEvent` 被下发至 Sink 组件时，它身上已经包含了充分的信息，我们可以将其转化为 SQL 或者特定消息格式的 Kafka 消息。在上文的架构图中我们可以看到有两种 Sink，一种是接入在 Table Pipeline 中的 TableSink，另外一种是 Processor 级别共用的 ProcessorSink。它们在系统中有不同的作用：\n\n- TableSink 作为一种 Table 级别的管理单位，缓存着要下发到 ProcessorSink 的数据，它的主要作用是方便 TiCDC 按照表为单位管理资源和进行调度\n- ProcessorSink 作为真实要与数据库或者 Kafka 建立连接的 Sink 负责 SQL/Kafka 消息的转换和同步\n\n我们再来看一看 ProcessorSink 到底如何转换这些行变更：\n\n- 如果下游是数据库，ProcessorSink 会根据 `RowChangedEvent` 中的 Columns 和 PreColumns 来判断它到底是一个 `Insert`、`Update` 还是 `Delete` 操作，然后根据不同的操作类型，将其转化为 SQL 语句，然后再将其通过数据库连接写入下游：\n\n```sql\n/*\n因为只有 Columns 所以是 Insert 语句。\n*/\nINSERT INTO TEST (NAME,AGE)\nVALUES ('Jack',20);\n\n/*\n因为既有 Columns 且有 PreColumns 所以是 Update 语句。\n*/\nUPDATE TEST\nSET AGE = 25\nWHERE NAME = 'Jack';\n```\n\n- 如果下游是 Kafka, ProcessorSink 会作为一个 [Kafka Producer](https://docs.confluent.io/platform/current/clients/producer.html) 按照特定的消息格式将数据发送至 Kafka。 以 [Canal-JSON](https://docs.pingcap.com/tidb/v6.0/ticdc-canal-json) 为例，我们上述的 Insert 语句最终会以如下的 JSON 格式写入 Kafka：\n\n```json\n{\n    \"id\": 0,\n    \"database\": \"test\",\n    \"table\": \"TEST\",\n    \"pkNames\": [\n        \"NAME\"\n    ],\n    \"isDdl\": false,\n    \"type\": \"INSERT\",\n    ...\n    \"ts\": 2,\n    \"sql\": \"\",\n    ...\n    \"data\": [\n        {\n            \"NAME\": \"Jack\",\n            \"AGE\": \"25\"\n        }\n    ],\n    \"old\": null\n}\n```\n\n这样，上游 TiDB 执行的 DML 就成功的被发送到下游系统了。\n\n## 结尾\n\n以上就是本文的全部内容。希望在阅读完上面的内容之后，读者能够对 TiCDC 是什么？为什么？怎么实现？这几个问题有一个基本的答案。","author":"江宗其","category":1,"customUrl":"ticdc-source-code-reading-1","fillInMethod":"writeDirectly","id":444,"summary":"本篇文章是 TiCDC 源码阅读系列文章的第一期，主要叙述了 TiCDC 的目的、架构和数据同步链路，旨在让读者能够初步了解 TiCDC，为阅读其他源码文章起到一个引子的作用。","tags":["TiCDC"],"title":"TiCDC 源码阅读（一）TiCDC 架构概览"}},{"relatedBlog":{"body":"## 引言\n\nTiCDC 作为 TiDB 的数据同步组件，负责直接从 TiKV 感知数据变更同步到下游。其中比较核心的问题是数据解析正确性问题，具体而言就是如何使用正确的 schema 解析 TiKV 传递过来的 Key-Value 数据，从而还原成正确的 SQL 或者其他下游支持的形式。本文主要通过对 TiDB Online DDL 机制原理和实现的分析，引出对当前 TiCDC 数据解析实现的讨论。 \n\n## 背景和问题\n\n数据同步组件是数据库生态中不可或缺的生态工具，比较知名的开源单机数据库 MySQL 就将数据同步作为 Server 能力的一部分，并基于 MySQL binlog 实现异步/半同步/同步的主从复制。由于 MySQL 悲观事务模型和表元数据锁的存在，我们总是可以认为 MySQL binlog 中存在因果关系的 data 和 schema 符合时间先后顺序的，即：\n\nNew data commitTs > New schema commitTs\n\n但是对于 TiDB 这种存储计算分离的架构而言，schema 的变更在存储层持久化，服务层节点作为多缓存节点，总是存在一个 schema 状态不一致的时间段。为了保证数据一致性和实现在线 DDL 变更，现有的分布式数据库大都采用或者借鉴了[Online, Asynchronous Schema Change in F1](https://static.googleusercontent.com/media/research.google.com/zh-CN//pubs/archive/41376.pdf) 机制。所以我们要回答的问题变成了，在 TiDB Online DDL 机制下，TiCDC 如何正确处理 data 和 schema 的对应关系，存在因果关系的 data 和 schema 是否仍然满足：\n\nNew data commitTs > New schema commitTs\n\n为了回答这个问题，我们首先需要先阐述原始的 F1 Online Schema Change 机制的核心原理，然后描述当前 TiDB Online DDL 实现，最后我们讨论在当前 TiCDC 实现下，data 和 schema 的处理关系和可能出现的不同的异常场景。\n\n## F1 Online Schema Change 机制\n\nF1 Online Schema Change 机制要解决的核心问题是，在单存储多缓存节点的架构下，如何实现满足数据一致性的 Online Schema 变更，如图 1 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktbgk8jj20x40u0wfz.jpg) \n\n<center>图 1: 单存储多缓存节点的架构下的 schema 变更</center>\n\n这里我们定义数据不一致问题为数据多余(orphan data anomaly)和数据缺失(integrity anomaly)，Schema 变更结束后出现数据多余和数据缺失我们就认为数据不一致了。这类系统的 schema 变更问题特点可以总结成以下 3 点：\n\n1. 一份 schema 存储，多份 schema 缓存\n\n2. 部分 new schema 和 old schema 无法共存\n\n3. 直接从 old schema 变更到 new schema 时，总是存在一个时间区间两者同时存在\n\n特点 1 和特点 3 是系统架构导致的，比较容易理解。特点 2 的一个典型例子是 add index，加载了 new schema 的服务层节点插入数据时会同时插入索引，而加载了 old schema 的服务层节点执行删除操作只会删除数据，导致出现了没有指向的索引, 出现数据多余。\n\nSchema 变更问题的特点 2 和特点 3 看起来是互相矛盾的死结，new schema 和 old schema 无法共存，但又必然共存。而 F1 Online Schema 机制提供的解决方案也很巧妙，改变不了结果就改变条件。所以该论文的解决思路上主要有 2 点，如图 2 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktc2oyjj210w0u0gog.jpg) \n\n<center>图 2: F1 Online DDL 解决方案</center>\n\n**1.** **引入共存的中间 schema 状态**，比如 S1->S2’->S2, S1 和 S2’ 可以共存，S2’ 和 S2 可以共存；\n\n**2.** **引入确定的隔离时间区间**，保证无法共存的 schema 不会同时出现；\n\n具体来讲：\n\n-  **引入共存的中间 schema 状态**\n\n因为直接从 schema S1 变更到 schema S2 会导致数据不一致的问题，所以引入了 delete-only 和 write-only 中间状态，从 S1 -> S2 过程变成 S1 -> S2+delete-only -> S2+write-only -> S2 过程，同时使用 lease 机制保证同时最多有 2 个状态共存。这时只需要证明**每相临的两个状态都是可以共存的**，保证数据一致性，就能推导出 S1 到 S2 变更过程中数据是一致的。\n\n-  **引入确定的隔离时间区间**\n\n定义 schema lease，超过 lease 时长后节点需要重新加载 schema，加载时超过 lease 之后没法获取 new schema 的节点直接下线，不提供服务。所以可以明确定义 2 倍 lease 时间之后，所有节点都会更新到下一个的 schema。\n\n### 引入共存的中间状态\n\n我们需要引入什么样的中间状态呢？那要看我们需要解决什么问题。这里我们仍然使用 add index 这个 DDL 作为例子，其他 DDL 细节可以查阅 [Online, Asynchronous Schema Change in F1](https://static.googleusercontent.com/media/research.google.com/zh-CN//pubs/archive/41376.pdf)。\n\n#### Delete-only 状态\n\n我们可以看到 old schema 是无法看到索引信息的，所以会导致出现删除数据，遗留没有指向的索引这种数据多余的异常场景，所以我们要引入的第一个中间状态是 delete-only 状态，赋予 schema 删除索引的能力。在 delete-only 状态下，schema 只能在 delete 操作的时候对索引进行删除，在 insert/select 操作的时候无法操作索引，如图 3 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktexqbtj21ir0u0di1.jpg) \n\n<center>图 3: 引入 delete-only 中间状态</center>\n\n原始论文对于 delete-only 的定义如下：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktf9usmj21xk0u079d.jpg) \n\n假设我们已经引入了明确的隔离时间区间（下一个小节会细讲），能保证同一时刻最多只出现 2 个 schema 状态。所以当我们引入 delete-only 状态之后，需要考虑的场景就变成:\n\n1. old schema + new schema(delete-only) \n\n2. new schema(delete-only)  + new schema\n\n- 对于场景 1，所有的服务层节点要么处于 old schema 状态，要么处于 new schema(delete-only) 状态。由于 index 只能在 delete 的时候被操作，所以根本没有 index 生成，就不会出现前面说的遗留没有指向的索引问题，也不会有数据缺失问题，此时数据是一致的。我们可以说 old schema 和  new schema(delete-only) 是可以共存的。\n\n- 对于场景 2，所有的服务层节点要么处于 new schema(delete-only) 状态，要么处于 new schema 状态。处于 new schema 状态的节点可以正常插入删除数据和索引，处于 new schema( delete-only)  状态的节点只能插入数据，但是可以删除数据和索引，此时存在部分数据缺少索引问题，数据是不一致的。\n\n引入 delete-only 状态之后，已经解决了之前提到的索引多余的问题，但是可以发现，处于 new schema( delete-only)  状态的节点只能插入数据，导致新插入的数据和存量历史数据都缺少索引信息，仍然存在数据缺失的数据不一致问题。\n\n#### Write-only 状态\n\n在场景 2 中我们可以看到，对于 add index 这种场景，处于 new schema( delete-only)  状态节点插入的数据和存量数据都存在索引缺失的问题。而存量数据本身数量是确定且有限的，总可以在有限的时间内根据数据生成索引，但是 new insert 的数据却可能随时间不断增加。为了解决这个数据缺失的问题，我们还需要引入第二个中间状态 write-only 状态，赋予 schema  insert/delete 索引的能力。处于 write-only 状态的节点可以 insert/delete/update 索引，但是 select 无法看到索引，如图 4 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktd1n7hj21e10u0q5b.jpg) \n\n<center>图 4: 引入 write-only 状态</center>\n\n原始论文中对于 write-only 状态的定义如下：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktdw1roj22o40u0agx.jpg) \n\n引入 write-only 状态之后，上述的场景 2 被切分成了场景 2‘ 和场景 3:\n\n2’: new schema(delete-only)  + new schema(write-only)\n\n3:  new schema(write-only) + new schema\n\n- 对于场景 2‘，所有的服务层节点要么处于 new schema(delete-only)  状态，要么处于 new schema(write-only) 。处于 new schema(delete-only)  状态的服务层节点只能插入数据，但是可以删除数据和索引，处于 new schema(write-only)  可以正常插入和删除数据和索引。此时仍然存在索引缺失的问题，但是由于 delete-only 和 write-only 状态下，索引对于用户都是不可见的，所以在用户的视角上，只存在完整的数据，不存在任何索引，所以内部的索引缺失对用户而言还是满足数据一致性的。\n\n- 对于场景 3，所有的服务层节点要么处于 new schema(write-only)  状态，要么处于 new schema。此时 new insert 的数据都能正常维护索引，而存量历史数据仍然存在缺失索引的问题。但是存量历史数据是确定且有限的，我们只需要在所有节点过渡到 write-only 之后，进行历史数据索引补全，再过渡到 new schema 状态，就可以保证数据和索引都是完整的。此时处于 write-only 状态的节点只能看到完整的数据，而 new schema 状态的节点能看到完整的数据和索引，所以对于用户而言数据都是一致的。\n\n#### 小节总结\n\n通过上面对 delete-only 和 write-only 这两个中间状态的表述，我们可以看到，在 F1 Online DDL 流程中，原来的单步 schema 变更被两个中间状态分隔开了。每两个状态之间都是可以共存的，每次状态变更都能保证数据一致性，全流程的数据变更也能保证数据一致性。\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktgltluj23y80o8adc.jpg) \n\n### 引入确定的隔离时间区间\n\n为了保证同一时刻最多只能存在 2 种状态，需要约定服务层节点加载 schema 的行为：\n\n1. 所有的服务层节点在 lease 之后都需要重新加载 schema；\n\n2. 如果在 lease 时间内无法获取 new schema，则下线拒绝服务；\n\n通过对服务层节点加载行为的约定，我们可以得到一个确定的时间边界，在 2*lease 的时间周期之后，所有正常工作的服务层节点都能从 schema state1 过渡到 schema state2, 如图 5 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktgfwycj21120u0dhd.jpg) \n\n<center>图 5: 最多 2*lease 时长后所有的节点都能过渡到下一个状态</center>\n\n### 中间状态可见性\n\n要正确理解原始论文的中间状态，需要正确理解中间状态的可见性问题。前面小节为了方便我们一直使用 add index 作为例子，然后表述 delete-only 和 write-only 状态下索引对于用户 select 是不可见的，但是 write-only 状态下，delete/insert 都是可以操作索引的。如果 DDL 换成 add column，那节点处于 write-only 状态时，用户 insert 显式指定新增列可以执行成功吗？答案是不能。\n\n总得来说，中间状态的 delete/insert 可见性是内部可见性，具体而言是服务层节点对存储层节点的可见性，而不是用户可见性。对于 add column 这个 DDL，服务层节点在 delete-only 和 write-only 状态下就能看到 new  column，但是操作受到不同的限制。对用户而言，只有到 new schema 状态下才能看到 new column，才能显式操作 new column，如图 6 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktcjki9j216i0u0jv0.jpg) \n\n<center>图 6: 中间状态可见性</center>\n\n为了清晰表述可见性，我们举个例子，如图 7 所示。原始的表列信息为 <c1>, DDL 操作之后表列信息为 <c1,c2>。\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktfttt1j21fq0u0jx6.jpg) \n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktegkboj21fi0u0n27.jpg) \n\n<center>图 7: 中间状态过渡</center>\n\n- 小图 (1)  中，服务层节点已经过渡到了场景 1，部分节点处于 old schema 状态，部分节点处于 new schema(delete-only) 状态。此时 c2 对用户是不可见的，不管是 insert<c1,c2> 还是 delete<c1,c2> 的显式指定 c2 都是失败的。但是存储层如果存在 [1,xxx] 这样的数据是可以顺利删除的，只能插入 [7] 这样的缺失 c2 的行数据。\n\n- 小图 (2)  中，服务层节点已经过渡到了场景 2，部分节点处于 new schema(delete-only) 状态，部分节点处于 new schema(write-only) 状态，此时 c2 对用户仍是不可见的，不管是 insert<c1,c2> 还是 delete<c1,c2> 的显式指定 c2 都是失败的。但是处于 write-only 状态的节点，insert [9] 在内部会被默认值填充成\t[9,0] 插入存储层。处于 delete-only 状态的节点，delete [9] 会被转成 delete [9,0]。\n\n- 小图 (3)  中，服务层所有节点都过渡到 write-only 之后，c2 对用户仍是不可见的。此时开始进行数据填充，将历史数据中缺失 c2 的行进行填充(实现时可能只是在表的列信息中打上一个标记，取决于具体的实现)。\n\n- 小图 (4)  中，开始过渡到场景 3，部分节点处于 new schema(write-only) 状态，部分节点处于 new schema 状态。处于 new schema(write-only) 状态的节点，c2 对用户仍是不可见的。处于 new schema 状态的节点，c2 对用户可见。此时连接在不同服务层节点上的用户，可以看到不同的的 select 结果，不过底层的数据是完整且一致的。\n\n### 总结\n\n上面我们通过 3 个小节对 F1 online Schema 机制进行了简要描述。原来单步 schema 变更被拆解成了多个中间变更流程，从而保证数据一致性的前提下实现了在线 DDL 变更。\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktb493ij23bn0u0gp4.jpg) \n\n对于 add index 或者 add column DDL 是上述的状态变更，对于 drop index 或者 drop column 则是完全相反的过程。比如 drop column 在 write-only 阶段及之后对用户都不可见了，内部可以正确 insert/delete，可见性和之前的论述完全一样。\n\n## TiDB Online DDL 实现\n\nTiDB Online DDL 是基于 F1 Online Schema 实现的，整体流程如图 8 所示：\n\n![img](https://tva1.sinaimg.cn/large/e6c9d24ely1gzvktdfwoej21400u0mys.jpg) \n\n<center>图 8 TiDB Online DDL 流程</center>\n\n简单描述如下：\n\n- TiDB Server 节点收到 DDL 变更时，将 DDL SQL 包装成 DDL job 提交到 TIKV  job queue 中持久化；\n\n- TiDB Server 节点选举出 Owner 角色，从 TiKV job queue 中获取 DDL job，负责具体执行 DDL 的多阶段变更；\n\n- DDL 的每个中间状态(delete-only/write-only/write-reorg)都是一次事务提交，持久化到 TiKV job queue 中;\n\n- Schema 变更成功之后，DDL job state 会变更成 done/sync，表示 new schema 正式被用户看到，其他 job state 比如 cancelled/rollback done 等表示 schema 变更失败；\n\n- Schema state 的变更过程中使用了 etcd 的订阅通知机制，加快 server 层各节点间 schema state 同步，缩短 2*lease 的变更时间。\n\n- DDL job 处于 done/sync 状态之后，表示该 DDL 变更已经结束，移动到 job history queue 中；\n\n详细的 TiDB 处理流程可以参见：[schema-change-implement.md](https://github.com/ngaut/builddatabase/blob/master/f1/schema-change-implement.md) 和 [TiDB ddl.html](https://pingcap.github.io/tidb-dev-guide/understand-tidb/ddl.html)\n\n## TiCDC 中 Data 和 Schema 处理关系\n\n前面我们分别描述了 TiDB Online DDL 机制的原理和实现，现在我们可以回到最一开始我们提出的问题：在 TiDB Online DDL 机制下，是否还能满足：\n\nNew data commitTs > New schema commitTs\n\n答案是否定的。在前面 F1 Online Schema 机制的描述中，我们可以看到在 add column DDL 的场景下，当服务层节点处于 write-only 状态时，节点已经能够插入 new column data 了，但是此时 new column 还没有处于用户可见的状态，也就是出现了 New data commitTs < New schema commitTs，或者说上述结论变成了：\n\nNew data commitTs > New schema(write-only) commitTs\n\n但是由于在 delete-only + write-only 过渡状态下，TiCDC 直接使用 New schema(write-only) 作为解析的 schema，可能导致 delete-only 节点 insert 的数据无法找到对应的 column 元信息或者元信息类型不匹配，导致数据丢失。所以为了保证数据正确解析，可能需要根据不同的 DDL 类型和具体的 TiDB 内部实现，在内部维护复杂的 schema 策略。\n\n在当前 TiCDC 实现中，选择了比较简单的 schema 策略，直接忽略了各个中间状态，只使用变更完成之后的 schema 状态。为了更好表述在 TIDB Online DDL 机制下，当前 TiCDC 需要处理的不同场景，我们使用象限图进行进一步归类描述。\n\n \n\n|                     | **Old schema** | **New schema** |\n| ------------------- | -------------- | -------------- |\n| **Old schema data** | 1              | 2              |\n| **New schema data** | 3              | 4              |\n\n \n\n- 1 对应 old schema 状态\n\n  此时 old schema data 和 old schema 是对应的***\\*；\\****\n\n- 4 对应 new schema public 及之后\n\n  此时 new schema data 和 new schema 是对应的；\n\n- 3 对应 write-only ~ public 之间数据\n\n  此时 TiCDC 使用 old schema 解析数据，但是处于 write-only 状态的 TiDB 节点已经可以基于 new schema  insert/update/delete 部分数据，所以 TiCDC 会收到 new schema data。不同 DDL 处理效果不同，我们选取 3 个常见有代表性的 DDL 举例。\n\n  - **add column**： 状态变更 absent -> delete-only -> write-only -> write-reorg -> public。由于 new schema data 是 TiDB 节点在 write-only 状态下填充的默认值，所以使用 old schema 解析后会被直接丢弃，下游执行 new schema DDL 的时候会再次填充默认值。对于动态生成的数据类型，比如 auto_increment 和 current timestamp，可能会导致上下游数据不一致。\n    \t\n\n\n  - **change column**：有损状态变更 absent -> delete-only -> write-only -> write-reorg -> public, 比如 int 转 double，编码方式不同需要数据重做。在 TiDB 实现中，有损 modify column 会生成不可见 new column，中间状态下会同时变更新旧 column。对于 TiCDC 而言，只会处理 old column 下发，然后在下游执行 change column，这个和 TiDB 的处理逻辑保持一致。\n\n\n  - **drop column**：状态变更 absent-> write-only -> delete-only -> delete-reorg -> public。write-only 状态下新插入的数据已经没有了对应的 column，TiCDC 会填充默认值然后下发到下游，下游执行 drop column 之后会丢弃掉该列。用户可能看到预期外的默认值，但是数据能满足最终一致性。\n\n\n- 2 对应直接从 old schema -> new schema\n\n  说明这类 schema 变更下，old schema 和 new schema 是可以共存的，不需要中间状态，比如 truncate table DDL。TiDB 执行 truncate table 成功后，服务层节点可能还没有加载 new schema，还可以往表中插入数据，这些数据会被 TiCDC 直接根据 tableid 过滤掉，最终上下游都是没有这个表存在的，满足最终一致性。\n\n## 总结\n\nTiCDC 作为 TiDB 的数据同步组件，数据解析正确性问题是保证上下游数据一致性的核心问题。为了能充分理解 TiCDC 处理 data 和 schema 过程中遇到的各种异常场景，本文首先从 F1 Online Schema Change 原理出发，详细描述在 schema 变更各个阶段的数据行为，然后简单描述了当前 TiDB Online DDL 的实现。最后引出在当前 TiCDC 实现下在 data 和 schema 处理关系上的讨论。","author":"黄国豪","category":1,"customUrl":"how-tidb-online-ddl-practice-in-ticdc","fillInMethod":"writeDirectly","id":354,"summary":"本文主要通过对 TiDB Online DDL 机制原理和实现的分析，引出对当前 TiCDC 数据解析实现的讨论。","tags":["TiCDC"],"title":"TiDB Online DDL 在 TiCDC 中的应用丨TiDB 工具分享"}},{"relatedBlog":{"body":"TiCDC 是一个通过拉取 TiKV 日志实现的 TiDB 增量数据同步工具，具有还原数据到与上游任意 TSO 一致状态的能力，同时提供开放数据协议，支持其他系统订阅数据变更。TiCDC 运行时是无状态的，借助 PD 内部的 etcd 实现高可用。TiCDC 集群支持创建多个同步任务，向多个不同的下游进行数据同步。\n\n在 4.0 之前，TiDB 提供 TiDB Binlog 实现向下游平台的近实时复制，在 TiDB 4.0 中，引入 TiCDC 作为 TiDB 变更数据的捕获框架。 TiCDC 首个 GA 版本随着 TiDB 4.0.6 正式发布，具备生产环境的运行能力，主要优势如下：\n\n![1-TiCDC系统架构图](https://img1.www.pingcap.com/prod/1_Ti_CDC_06ccc8a0f8.png)\n\n<div class=\"caption-center\">TiCDC 系统架构图</div>\n\n- **数据高可用**：TiCDC 从 TiKV 获取变更日志，意味着只要 TiKV 具备高可用就能保证数据的高可用，遇到 TiCDC 全部异常关闭的极端情况，后续启动还能正常获取数据。\n\n- **水平扩展**：支持组建多 TiCDC 节点集群，将同步任务均匀地调度到不同节点上，面对海量数据时，可以通过添加节点解决同步压力过大的问题。\n\n- **自动故障转移**：当集群中的一个 TiCDC 节点意外退出时，该节点上的同步任务会自动调度到其余的 TiCDC 节点上。\n\n- **支持多种下游系统和输出多种格式**：目前已经支持兼容 MySQL 协议的数据库、Kafka 和 Pulsar 分布式流处理系统，支持输出的格式有 [Apache Avro](http://avro.apache.org/)，[Maxwell](http://maxwells-daemon.io/) 和 [Canal](https://github.com/alibaba/canal)。 \n\n## 应用场景\n\n### 两中心主备\n\n数据库作为企业 IT 的核心，在稳定运行的基础之上，数据库的容灾建设成为保障业务连续性的前提条件。\n\n综合考量业务关键性、成本等因素，部分用户希望核心数据库只需要完成主备站点的容灾即可，利用 TiCDC 构建 TiDB 主备站点的容灾方案成为理想之选。该方案基于 TiCDC 的数据同步功能，可以适配两个中心长距离间隔、网络延迟较大的场景，进行两个数据中心 TiDB 集群之间的数据单向同步，保障事务的最终一致性，实现秒级 RPO 。\n\n### 环形同步与多活\n\n利用 TiCDC 实现三个 TiDB 集群之间的环形同步，构建 TiDB 多中心的容灾方案。当一个数据中心的机柜发生意外掉电，可以把业务切换到另一个数据中心的 TiDB 集群，实现事务的最终一致性和秒级 RPO。为了分担业务访问的压力，在应用层可以随时切换路由，将流量切换到目前负载较小的 TiDB 集群提供服务，实现负载均衡，在满足数据高可用的同时提升容灾能力。\n\n![2-TiCDC环形同步拓扑图](https://img1.www.pingcap.com/prod/2_Ti_CDC_c1d2dceef1.png)\n\n<div class=\"caption-center\">TiCDC环形同步拓扑图</div>\n\n### 数据订阅\n\nTiCDC 为下游数据消费端提供实时、高吞吐、稳定的数据订阅服务，通过开放数据协议（Open Protocol ）与 MySQL、Kafka、Pulsar、Flink、Canal、Maxwell 等多种异构生态系统对接，满足用户在大数据场景中对各类数据的应用与分析需求，广泛适用于日志收集、监控数据聚合、流式数据处理、在线和离线分析等场景。\n\n## 最佳实践\n\n### 小红书\n\n小红书是年轻人的生活方式平台，用户可以通过短视频、图文等形式记录生活点滴，分享生活方式，并基于兴趣形成互动。截至到 2019 年 10 月，小红书月活跃用户数已经过亿，并持续快速增长。\n\n小红书在报表分析、大促实时大屏、物流仓储、电商数据中台、内容安全审核分析等多个场景使用 TiDB 承载核心业务。在内容安全审核分析场景，上游 TiDB 承载安全审核数据的实时记录，由线上应用直接写入，实现实时数据的监控和分析。\n\n在审核数据分析的业务流程中，通过 TiCDC 抽取 TiDB 的实时流数据，接到下游 Flink 进行实时计算聚合，计算结果再次写入 TiDB，用于审核数据的分析、人工效率的分析和管理等。小红书通过调用 TiCDC 内部 API（[Sink Interface](https://pkg.go.dev/github.com/pingcap/ticdc@v0.0.0-20200914115832-993bfabc4696/cdc/sink?tab=doc#Sink)） 自定义 Sink，使用 Canal 协议发送数据到 Flink，对接已有业务系统，显著降低业务系统的改造成本。TiCDC 高效的数据同步，对异构数据生态的支持，为小红书业务数据的实时处理奠定了坚实基础。\n\n### 汽车之家\n\n汽车之家是全球访问量最大的汽车网站，致力于通过产品服务、数据技术、生态规则和资源为用户和客户赋能，建设“车媒体、车电商、车金融、车生活” 4 个圈。\n\nTiDB 在汽车之家已经稳定运行两年多的时间，承载了论坛回复，资源池，好友关系等重要业务。在 2020 年 818 大型促销活动中，汽车之家采用 TiDB 两地三中心的方案，为秒杀、红包、抽奖、摇一摇等场景提供全方位的数据保障，使用 TiCDC 将 TiDB 集群数据实时同步至下游的 MySQL 数据库，作为故障应急的备份，实现业务容灾能力的提升。TiCDC 数据同步的延迟在秒级别，很好地满足了线上促销类业务的实时性要求。\n\n智能推荐是汽车之家的一个重要业务，资源池是智能推荐的底层存储。资源池接收和汇聚各类资讯信息，进行数据加工后供首页推荐、产品展示、搜索等业务前台做推荐展示。资源池前期使用 MySQL 作为存储层，采用 MySQL Binlog 传入 ElasticSearch，满足检索场景需求。由于 MySQL 出现性能和容量的瓶颈，切换到 TiDB 之后，汽车之家采用 TiCDC 做异构数据的同步，替换原有的 MySQL Binlog 方案。TiCDC 高可用、低延迟、支持大规模集群等特性，为业务提供稳定的数据支撑。汽车之家基于 TiCDC 开发了日志数据输出到 Kafka 接口，实现海量异构数据的同步处理，目前已经上线并稳定运行两个多月。\n\n### 海尔智家\n\n海尔智家 APP 是海尔发布的移动端官方体验交互入口，为全球用户提供智慧家庭全流程服务、全场景智家体验与一站式智家定制方案。\n\n海尔智家的 IT 技术设施构建在阿里云上，核心业务要求数据库支持 MySQL 协议，在满足强一致分布式事务的基础上，提供灵活的在线扩展能力，并且可以与各类大数据技术生态紧密融合，TiDB 4.0 成为海尔智家的理想之选。\n\n利用 TiDB 增量数据同步工具  TiCDC 将用户信息和生活家信息同步到 ElasticSearch，提供近实时的搜索功能。目前用户表数据近千万，数据量达到 1.9G，Kafka 日消费消息量在 300万左右。此外，TiCDC 为智能推荐的大数据服务提供稳定、高效的数据同步，基于统一的 TiCDC Open Protocol 行级别的数据变更通知协议，极大方便了不同部门的数据解析需求，目前智能推荐的功能正在开发中。\n\n### 知乎\n\n知乎是中文互联网综合性内容平台，以“让每个人高效获得可信赖的解答”为品牌使命和北极星。\n\n知乎在首页个性化内容推荐、已读服务等场景中使用 TiDB 作为核心数据库，通过 TiCDC Open Protocol 输出日志到 Kafka，进行海量的消息处理。随着业务量级的增长，在使用的过程中遇到了诸多因 Kafka 架构和历史版本实现上的限制而引发的问题。考虑到 Pulsar 对原生跨地域复制（GEO-Replication）的支持同知乎未来基础设施云原生化的方向更加契合，知乎开始在一些业务中使用 Pulsar 替换 Kafka 。\n\n知乎对 TiCDC 的核心模块进行了一系列开发工作（https://github.com/pingcap/ticdc/pull/751， https://github.com/pingcap/ticdc/pull/869），把 TiCDC Sink 与 Pulsar 进行对接，实现 TiCDC 的数据同步到 Pulsar。借助 Pulsar 所具有的 GEO-Replication 功能，可以为 TiCDC 的消费者带来地理位置无关的变更事件订阅能力。Pulsar 集群的快速节点扩容、故障的快速恢复能力可以为 TiCDC 事件的消费方提供更优的数据实时性保障。\n\n从前期业务的实践来看，Pulsar 与 TiCDC 的应用取得了理想效果。知乎将推动各项业务从 Kafka 向 Pulsar 进行全面的迁移，未来也将应用 Pulsar 到跨集群同步 TiDB 数据的场景下。\n\n## 体验 TiCDC\n\n大家可以通过 TiUP （[部署文档](https://docs.pingcap.com/zh/tidb/stable/manage-ticdc#%E4%BD%BF%E7%94%A8-tiup-%E9%83%A8%E7%BD%B2%E5%AE%89%E8%A3%85-ticdc)）上快速部署 TiCDC，通过 `cdc cli` 创建同步任务，将实时写入同步到下游的 TiDB 中，或者下游 Pulsar 中，又或者下游 Kafka 中（[操作文档](https://docs.pingcap.com/zh/tidb/stable/manage-ticdc#%E7%AE%A1%E7%90%86%E5%90%8C%E6%AD%A5%E4%BB%BB%E5%8A%A1-changefeed)）\n\n对于 4.0.6 GA 之前版本的用户，请参考[升级文档](https://docs.pingcap.com/zh/tidb/dev/manage-ticdc#%E4%BD%BF%E7%94%A8-tiup-%E5%8D%87%E7%BA%A7-ticdc)。\n\n## 致谢\n\n感谢[所有 TiCDC 的贡献者](https://github.com/pingcap/ticdc/graphs/contributors)，TiCDC 能够走到 GA 离不开每一位贡献者的努力！","author":"PingCAP","category":1,"customUrl":"ticdc-ga","fillInMethod":"writeDirectly","id":219,"summary":"本文将向大家介绍 TiCDC，一个通过拉取 TiKV 日志实现的 TiDB 增量数据同步工具，具有还原数据到与上游任意 TSO 一致状态的能力，同时提供开放数据协议，支持其他系统订阅数据变更。","tags":["TiCDC","TiDB"],"title":"TiCDC 首个 GA 版本发布，特性与场景全揭秘丨TiDB 工具"}}]}}},
    "staticQueryHashes": ["1327623483","1820662718","3081853212","3430003955","3649515864","4265596160","63159454"]}