github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/docs/design/2020-04-13-ticdc-auto-gc-safepoint-1-cn.md (about)

     1  # 自适应 GC safepoint 设计
     2  
     3  - Author(s): [overvenus](https://github.com/overvenus) (Taining Shen)
     4  - Last updated: 2020-04-13
     5  
     6  ## 概述
     7  
     8  本文档提出了一种新的 GC safepoint 前进机制,可以根据 TiDB 集群中的组件的要求自适应调整 GC safepoint。
     9  
    10  ## 问题和设计目标
    11  
    12  目前 TiDB 的一些服务(比如 CDC,Mydumper 和 BR)和 GC safepoint 紧密相关,一旦 GC safepoint 前进过快就会导致这些服务的不可用。GC safepoint 也不能前进过慢(默认为当前时间 - 10min),否则就有可能导致 TiKV 堆积过多历史数据,影响到在线业务。可见,当集群中存在 CDC,Mydumper 或者 BR 时,对 GC safepoint 的调整需要在满足这些服务要求的同时尽可能地快。
    13  
    14  ### 目前的实现
    15  
    16  目前 GC 的方案见 [分布式 GC 方案](https://docs.google.com/document/d/1tQVC7QlsfkAO4X-cT4kDjSfUvmEmwSUDxpe0lZBVwyU/edit) 和 [PD 现有接口](https://github.com/pingcap/kvproto/blob/b8bc94dd8a3690423ce8051d4ea0a758aff5d2bf/proto/pdpb.proto),这里具体说下 GC 的 safepoint 更新的机制:
    17  
    18  1. TiDB 中定时触发更新 safepoint
    19  2. TiDB 通过调用 pd.UpdateGCSafepoint 更新保存在 PD 上的 GC safpoint
    20  3. TiKV 定时调用 pd.GetGCSafepoint 获取保存在 PD 上的 GC safepoint
    21  4. TiKV 根据 GC safepoint 删除本地数据
    22  
    23  通过观察上述步骤可以发现,
    24  
    25  1. GC safepoint 保存了两份,分别在 TiDB 和 PD,但是**真正**会影响到 TiKV 的 GC safepoint 保存在 PD 上。
    26  2. GC safepoint 的前进由 TiDB 驱动,但是 TiDB 没有考虑到其他服务。
    27  
    28  了解清楚现状后,我们再来看自适应 GC safepoint。
    29  
    30  ## 自适应 GC safepoint
    31  
    32  整个方案就两点:1. GC safepoint 以 PD 为准,2. PD 提供接口设置对 GC safepoint 的限制。
    33  
    34  GC safepoint 以 PD 为准是因为,
    35  
    36  1. 尽量靠近现在的实现
    37  2. PD 非常适合存储集群元数据
    38  3. txn kv 在没有 TiDB 情况也能使用
    39  
    40   PD 提供接口设置对 GC safepoint 的限制是因为,
    41  
    42  1. 提供一种手段让外部服务按需设置 GC safepoint 的限制
    43  
    44  ### 细节
    45  
    46  PD 添加一个新的 RPC: SetGCSafePointLimit。
    47  
    48  ```
    49  rpc SetGCSafePointLimit(SetGCSafePointLimitRequest) returns (SetGCSafePointLimitResponse) {}
    50  message SetGCSafePointLimitRequest {
    51     RequestHeader header = 1;
    52     // UUID v4
    53     bytes uuid = 2;
    54     uint64 safe_point = 3;
    55     uint64 ttl = 4;
    56  }
    57  message SetGCSafePointLimitResponse {
    58     ResponseHeader header = 1;
    59  }
    60  ```
    61  
    62  - UUID 是为了 1. 能让多个服务同时对 GC safepoint 做限制,2. 外部服务可以不断前进自己的 limit。
    63  - TTL 是为了确保在外部异常退出的时不会过分阻塞 GC safepoint 的前进。
    64  - TTL 有最大值的限制,可在 PD 侧配置,如果请求 TTL 超过最大值则回复报错
    65  - PD 在 UpdateSafepoint 时,需要确保前进后的safepoint 满足所有的 limit 要求。
    66  - 如果 SetGCSafePointLimit 请求中的 safe_point 已经小于当前 PD 的 safepoint 则回复报错
    67  
    68  新设计完整 GC 的 safepoint 更新的机制:
    69  
    70  1. TiDB 中定时触发调用 pd.UpdateGCSafepoint
    71  2. PD 根据各个 limit 尝试前进 GC safepoint
    72  3. TiDB 将 PD 返回的 safepoint 保存到本地(兼容性要求)
    73  4. TiKV 定时调用 pd.GetGCSafepoint 获取保存在 PD 上的 GC safepoint
    74  5. TiKV 根据 GC safepoint 删除本地数据
    75  
    76  min(safe_point, limit) 可以通过扩展现有的 GetGCSafepoint RPC 接口获得
    77  
    78  ```
    79  
    80  message GetGCSafePointResponse {
    81     RequestHeader header = 1;
    82     uint64 safe_point = 2;
    83     uint64 min_safe_point_limit = 3;
    84  }
    85  ```
    86  
    87  ### 用法举例
    88  
    89  以 CDC 为例,CDC 服务中有一个 checkpoint ts 的概念,它的通常只落后于当前时间 1s~1min 之间,我们需要确保 checkpoint ts > GC safepoint,同时我们预计 CDC 意外停止服务后,最多需要 12 小时恢复服务。那么我们可以每隔 10s 向 PD 发生同下请求
    90  
    91  ```
    92  message SetGCSafePointLimitRequest {
    93     bytes uuid = CDC_UUID;
    94     uint64 safe_point = checkpoint_ts;
    95     uint64 ttl = 12 * 60 * 60; // 12h
    96  }
    97  ```