TiDB 組件 GC 原理及常見問題

導讀

本文詳細介紹了 TiDB 的 Garbage Collection(GC)機制及其在 TiDB 組件中的實現原理和常見問題排查方法。 TiDB 底層使用單機存儲引擎 RocksDB,並通過 MVCC 機制,基於 RocksDB 實現了分佈式存儲引擎 TiKV,以支持高可用分佈式事務。 GC 過程旨在清理舊數據,減少其對性能的影響,主要包括四個步驟: 計算 GC safepoint、解析鎖(Resolve locks)、連續範圍數據刪除(Delete ranges)和同步 GC safepoint 至集羣其他組件。 文章還講述瞭如何定位 GC leader、監控 GC 狀態、以及處理 GC 過程中遇到的常見問題。

TiDB 底層使用的是單機存儲引擎 RocksDB, 爲了實現分佈式事務接口,TiDB 又採用 MVCC 機制,基於 RocksDB 實現了高可用分佈式存儲引擎 TiKV。 也就是當新寫入(增刪改)的數據覆蓋到舊數據時,舊數據不會被替換掉,而是與新寫入的數據同時保留,並以時間戳來區分版本。 當這些歷史版本堆積越來越多時,就會引出一系列問題,最常見的便是讀寫變慢。 TiDB 爲了降低歷史版本對性能的影響,會定期發起 Garbage Collection( https://docs-archive.pingcap.com/zh/tidb/v7.2/garbage-collection-overview GC)清理不再需要的舊數據。

 上一篇文章 中,我們介紹了 MVCC 版本堆積相關原理及排查手段,當我們發現 MVCC 版本堆積已經對當前集羣讀寫產生了性能影響時,則需要檢查當前集羣 GC 的狀態及相關參數是否需要進行調整。

本文我們將重點介紹 TiDB 組件中 GC 的相關原理及常見排查手段。由於篇幅原因,TiKV 側的 GC 相關內容我們將在另一篇文章中獨立介紹。

GC leader

通過對 TiDB 分佈式事務( https://tidb.net/blog/7730ed79 )實現的瞭解,我們知道 TiDB 集羣具體的數據存儲在 TiKV 上,集羣的元數據信息存在 PD 上,TiDB 要做數據舊版本的回收,則需要有個類似 GC worker 的角色從 PD 拿到元數據信息然後對 TiKV 中的數據做垃圾回收工作。這個角色目前我們放在 TiDB 中,一個就夠,所以我們藉助 PD 維護選舉出一個 GC leader 的角色,來統一協調整個集羣的 GC 工作。

GC leader 是 TiDB 中負責推動集羣 GC 工作的一個協程(goroutinue), 一個 TiDB 集羣中,同一時刻有且只有一個 TiDB 上會有這個 GC leader 角色。

常見排查指導

通常如果懷疑係統 GC 狀態可能存在異常,我們可以從 gc leader 所報的日誌中查看當前 GC 的詳細狀態。

  • 如何查找 GC leader 在哪個 tidb 上?
mysql> select variable_name,variable_value from mysql.tidb where variable_name = "tikv_gc_leader_desc"\G;
*************************** 1. row ***************************
 variable_name: tikv_gc_leader_desc
variable_value: host:172-16-120-219, pid:3628952, start at 2024-01-17 16:34:58.022047311 +0800 CST m=+9.910349289
1 row in set (0.00 sec)
  • 找到 gc leader 對應的 tidb 實例,在日誌中 grep "gc_worker" 關鍵字即可看到當前 GC 的整體運行狀態。關於日誌中字段的具體含義,我們可以在後面的章節會詳細展開介紹。
tidb@172-16-120-219:~/shirly/tiup$ ps aux | grep 3628952
tidb     3592616  0.0  0.0   8160  2456 pts/2    S+   15:12   0:00 grep --color=auto 3628952
tidb     3628952  8.2  0.2 10914336 1065168 ?    Ssl  Jan17 4158:25 bin/tidb-server -P 4005 --status=10080 --host=0.0.0.0 --advertise-address=127.0.0.1 --store=tikv --initialize-insecure --path=127.0.0.1:2379,127.0.0.1:2381,127.0.0.1:2383 --log-slow-query=/DATA/disk4/shirly/tiup/tidb-deploy/tidb-4005/log/tidb_slow_query.log --config=conf/tidb.toml --log-file=/DATA/disk4/shirly/tiup/tidb-deploy/tidb-4005/log/tidb.log
tidb@172-16-120-219:~/shirly/tiup$ grep "gc_worker" /DATA/disk4/shirly/tiup/tidb-deploy/tidb-4005/log/tidb.log  | head
[2024/02/02 19:29:59.062 +08:00] [INFO] [gc_worker.go:1073] ["[gc worker] start resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446541678411803] [concurrency=3]
[2024/02/02 19:29:59.291 +08:00] [INFO] [gc_worker.go:1095] ["[gc worker] finish resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446541678411803] [regions=1236]
[2024/02/02 19:30:59.065 +08:00] [INFO] [gc_worker.go:1073] ["[gc worker] start resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446557407051824] [concurrency=3]
[2024/02/02 19:30:59.283 +08:00] [INFO] [gc_worker.go:1095] ["[gc worker] finish resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446557407051824] [regions=1236]
[2024/02/02 19:31:59.060 +08:00] [INFO] [gc_worker.go:1073] ["[gc worker] start resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446573135691804] [concurrency=3]
[2024/02/02 19:31:59.297 +08:00] [INFO] [gc_worker.go:1095] ["[gc worker] finish resolve locks"] [uuid=6345a3cad480026] [safePoint=0] [try-resolve-locks-ts=447446573135691804] [regions=1236]
[2024/02/02 19:32:59.072 +08:00] [INFO] [gc_worker.go:347] ["[gc worker] starts the whole job"] [uuid=6345a3cad480026] [safePoint=447446510221131776] [concurrency=3]
[2024/02/02 19:32:59.074 +08:00] [INFO] [gc_worker.go:1073] ["[gc worker] start resolve locks"] [uuid=6345a3cad480026] [safePoint=447446510221131776] [try-resolve-locks-ts=447446588864331831] [concurrency=3]
[2024/02/02 19:32:59.305 +08:00] [INFO] [gc_worker.go:1095] ["[gc worker] finish resolve locks"] [uuid=6345a3cad480026] [safePoint=447446510221131776] [try-resolve-locks-ts=447446588864331831] [regions=1236]
[2024/02/02 19:33:59.057 +08:00] [INFO] [gc_worker.go:307] ["[gc worker] there's already a gc job running, skipped"] ["leaderTick on"=6345a3cad480026]

TiDB GC 整體流程及常見問題

TiDB GC 整體流程主要分四個步驟,本章我們將逐一展開介紹。

目前我們 TiDB 側的 GC 流程主要分爲四個步驟:

  • 計算 GC safepoint, 即 TiDB GC 時需要知道具體刪除哪個時間點之前的舊版本。
  • 清理 GC safepoint 之前事務留下的鎖,即舊版本數據在被清理前,需要明確殘留鎖所在事務的狀態。
  • Delete-ranges 連續範圍數據刪除,即對於 truncate table 等這類在 TiKV 中連續保存的數據,直接在此階段進行物理刪除以優化性能。
  • 將最新 safepoint 同步到集羣其他組件(TiKV)

TiDB 中 GC worker 以上行爲的發生頻率我們可以在 grafana 監控中 tikv-details->GC->TiDB GC worker actions 看到。

下面我們逐一介紹每個步驟的原理、相關監控、配置及常見問題。

Step 1 計算 GC safepoint

根據 文章 對數據寫入的簡單介紹,我們知道當對同一個 key 進行多次增刪改後,會在 raftstore 層留下所有的歷史版本,隨着這些版本的堆積,整體的讀寫性能將受到影響。TiDB 會定期觸發 GC 工作對這些歷史版本進行回收,那每次 GC 具體回收哪些舊版本數據呢?TiDB 在每次發生 GC 時,都會根據當前配置參數計算出一個 safepoint , 來決定具體回收哪些舊版本數據。

GC safepoint 的定義

  • Safepoint 是一個時間戳,對應一個具體的物理時間。
  • TiDB GC 會保證 ts >safepoint 的所有快照數據的安全性。

如上圖,當前 key 一共有四個版本,

  • 如果 gc safepoint 是 5:00, 則 GC 後只會保留 key_4:00 這條數據。
  • 如果 gc safepoint 是 2:30, 則 GC 後會保留 key_4:00,key_3:00,key_02:00 這三條數據。以確保讀 2:30 這一時刻的快照時,能讀到 key_02:00 這條數據。

另外,當前系統的 gc safepoint 我們也可以在系統表 mysql.tidb 中看到:

GC safepoint 的計算過程及常見問題

瞭解了 GC safepoint, 我們知道其對集羣數據的安全性非常重要,算錯了 safepoint, 就可能將還需要的舊版本數據提前永久刪除掉。所以在計算 safepoint 的時候,我們考慮到了多種情況。

以下是當前 gc worker 計算 safepoint 的主要過程:

1. 根據 GC lifetime 配置計算

GC lifetime ( https://docs.pingcap.com/tidb/stable/system-variables#tidb_gc_life_time-new-in-v50 )的定義:這個變量用於指定每次 GC 時需要保留的數據時限,默認爲 10 分鐘,即一般情況下,只保證十分鐘以內的數據快照安全性即可。

GC safepoint = Current time - GC lifetime (10min by default)
  • 常見問題

當我們調整 gc lifetime 時,比如調大 gc lifetime 時,在 lifetime 符合要求之前會跳過幾次 GC,相關 gc_worker 的日誌如下

tidb_172.26.55.107_4000.log:[2024/01/16 09:06:52.689 +08:00] [Info] [gc_worker.go:1613] ["[gc worker] sent safe point to PD"] [uuid=6342b1728dc000d] ["safe point"=447035326078648378]
tidb_172.26.55.107_4000.log:[2024/01/16 09:15:11.099 +08:00] [Info] [gc_worker.go:731] ["[gc worker] there's another service in the cluster requires an earlier safe point. gc will continue with the earlier one"] [uuid=6342b1728dc000d] [ourSafePoint=447035555467755520] [minSafePoint=447035326078648378]

對於這種情況符合預期,無緒介入。

2. 檢查長時間運行且未提交的事務

檢查當前集羣所有 session 裏面中,是否存在未提交的事務,且該事務 begin 時間早於上一步算出來的 gc safepoint

GC safepoint = min(GC safepoint,min_start_ts among all sessions)

也就是說,gc safepoint 不應晚於當前正在執行中的事務的開始時間。

爲了降低長時間運行的未提交事務對 GC 的影響,我們在 v6.1.0 中引入了參數 tidb_gc_max_wait_time( https://docs.pingcap.com/tidb/dev/system-variables#tidb_gc_max_wait_time-new-in-v610 ) (默認 24 小時),也就是當某個事務執行時間超過 24 小時後,該事務不會再卡住 gc safepoint 的推進。

  • 常見問題

GC 在這一步卡住,可以從 gc_worker 日誌中看到具體卡住的事務詳情:

[gc_worker.go:359] ["[gc worker] gc safepoint blocked by a running session"] [uuid=609099af5940005] [globalMinStartTS=437144990507073574] [safePoint=2022/11/04 23:29:59.969 +08:00]

Workaround: 檢查 processlist 定位卡住的那個事務,諮詢業務側是否可以對該事務進行清理:

select * from INFORMATION_SCHEMA.CLUSTER_PROCESSLIST where TIMESTAMPDIFF(minute, now(), concat("2021-", substring_index(txnstart, "(", 1) )) < -10;

3. 檢查當前工具需要保留的快照版本

在實際業務集羣中,用戶可能使用了 CDC/BR 等備份工具,這些備份工具可能需要更早的一個快照進行備份,也就意味着這部分數據不能被 GC 掉。

GC safepoint = min(GC safepoint, min_service_gc_safe_points)
  • 常見問題

如果 GC 被這部分卡住了,一般可以通過 gc_worker 的日誌類似如下:

[2022/02/24 17:18:01.444 +05:30] [INFO] [gc_worker.go:411] ["[gc worker] gc safepoint blocked by a running session"] [uuid=5f56cf29bac008e] [globalMinStartTS=431397745740742701] [safePoint=431398894023213056]
[2022/02/24 17:18:01.450 +05:30] [INFO] [gc_worker.go:581] ["[gc worker] there's another service in the cluster requires an earlier safe point. gc will continue with the earlier one"] [uuid=5f56cf29bac008e] [ourSafePoint=431397745740742701] [minSafePoint=431337131664474119] 

在確認了是這一步卡住的後,就可以通過 PD 查看一下具體是哪個服務卡住了 GC 並介入處理了。

// 通過 pd-ctl 查看 service_gc_safe_points 下面的服務需要的最舊快照對應的 safepoint
tidb@172-16-120-219:~/shirly/tiup$  tiup ctl:v7.5.0 pd  -u http://127.0.0.1:2379  service-gc-safepoint
Starting component `ctl`: /home/tidb/.tiup/components/ctl/v7.5.0/ctl pd -u http://127.0.0.1:2379 service-gc-safepoint
{
  "service_gc_safe_points": [
    {
      "service_id": "gc_worker",
      "expired_at": 9223372036854775807,
      "safe_point": 447873652897873920 // 這個是在當前這一步上傳的 gc safepoint.
    },
    {
      "service_id": "ticdc-default-157...",
      "expired_at": 9223372036854645313,
      "safe_point": 447873653919043430
    }
  ],
  "gc_safe_point": 447873652897873920 // 這個是 TiDB GC 最後一步上傳的 safepoint, 用於通知 tikv 用。
}
// 計算每個 service safepoint 實際對應的時間。
tidb@172-16-120-219:~/shirly/tiup$ tiup ctl:v7.5.0 pd  -u http://127.0.0.1:2379  tso 447873652897873920
Starting component `ctl`: /home/tidb/.tiup/components/ctl/v7.5.0/ctl pd -u http://127.0.0.1:2379 tso 447873652897873920
system:  2024-02-21 15:59:59.055 +0800 CST
logic:   0
tidb@172-16-120-219:~/shirly/tiup$ tiup ctl:v7.5.0 pd  -u http://127.0.0.1:2379  tso 447873653919043430
Starting component `ctl`: /home/tidb/.tiup/components/ctl/v7.5.0/ctl pd -u http://127.0.0.1:2379 tso 447873653919043430
system:  2024-02-21 16:00:02.95 +0800 CST
logic:   118630

Step 2 : Resolve locks

爲什麼要清理鎖

在有了 gc safepoint 之後,意味着在本輪 GC 中,我們在保證所有 tso >= safepoint 版本的快照安全性的基礎上,可以開始刪除舊版本了, 那麼在刪除舊版本之前,如果遇到了鎖怎麼辦呢?根據我們之前對分佈式事務的理解,用戶在 commit 一個事務之後,TiKV 內部還是有可能留下鎖的,而這些鎖的提交狀態則是存在 primary-key 上,試想以下情況:

  • 事務 1:
  • 事務 ID 即 start_ts 是 t1, commit_ts 爲 t2。
  • 更新 A,B,C 三個 key, 客戶端 commit 且返回成功。
  • 當前分佈式事務 primary_key  A , 也就是事務的狀態存在 A 上 。
  • B,C 上有當前 t1 所在事務的殘留 lock 
  • 事務 2:
  • 事務 ID 即 start_ts 爲 t3, commit_ts 爲 t4。(t1<t2<t3<t4)。
  • 更新 A, D 兩個 key, 客戶端 commit 且返回成功。
  • A,**D** 新版本寫入成功,無殘留鎖。

現在假設我們算出來的 GC safepoint 是 t4 , 也就是保證能讀到 t4 這一時刻的快照數據一致即可。對於 A , 當前 t4 可見的數據爲 A_t4=>t3,A_t3=>12, 也就是 A 在 t4 這個時刻快照讀到的數據是 12。

對於比這個版本更舊的版本 A_t2=>t1 我們認爲在當前 gc safepoint 下是可以刪除的。那我們可以直接刪除 A_t2=>t1 這個版本嗎?

假設我們直接刪除,刪除之後,如果用戶要讀 t4 這個快照裏面 B 的值,發現 B 上有個指向 (A,t1) 的這個 lock, 我們開始從 A 上確認事務 t1 的狀態,但是在 TiKV 中找不到 (A,t1) 這個事務,也就無法確認其狀態。

以上,就是我們爲什麼要在真正清理舊版本數據之前,要先對 gc safepoint 之前啓動的事務所在殘留鎖進行清理的原因,這個過程我們定義爲 Resolve locks。

Resolve locks 具體過程

知道了 Resolve locks 的原因後,我們很容易就理解,resolve locks 這一步簡單理解,就是對 tikv 中的 Lock CF 進行掃描,並清理掉 lock.ts <= gc safepoint 的鎖即可。在實際操作中,我們操作步驟如下:

  1. 將請求發給每個 region 的 leader 獲取到 lock
  2. 根據 lock 狀態,逐個 resolve lock:
  3. 向 PD 定位當前 lock 裏面 primary-key 所在的 region 信息
  4. 向對應的 TiKV 發送獲取當前 (primary-key,事務 ID ) 對應的事務狀態
  5. 根據 (primary-key,事務 ID) 對應的狀態:
  • 事務已提交,向 tikv 提交 lock。
  • 事務已 rollback, 向 tikv 發送回滾該 lock 的消息。

相關配置

既然是按照 region 查找鎖並進行清理,這個過程完全可以是併發的,針對這一步,目前 TiDB 提供了以下參數:

相關監控

在 grafana 上 tikv-details/gc/resolveLocks Progress 面板中,可以看到這一步驟以 region 爲單位的執行進度。

常見問題

這一步是否可能卡住 GC?

一般的 resolveLocks 不應該卡住 GC,當這一步驟出現問題時,數據可能存在一致性問題,如果數據比較重要的話,應立即聯繫官方協助恢復。同樣的,這一步驟出現問題需要通過 gc worker 的日誌進行判斷,可以參考 GC leader 章節的方式定位相關日誌。

歷史上 v5.1.3 有個 bug( https://github.com/pingcap/tidb/issues/26359 )會導致數據一致性的問題而導致 Resolve Locks 失敗。錯誤日誌類似長這樣:

[gc_worker.go:621]["[gc worker] resolve locks returns an error"]..

Resolve locks 對性能的影響

對於比較空閒的集羣,GC 期間 resolve locks 是會對 TiKV 產生性能影響的,主要原因爲 resolve lock 需要對集羣中所有 region 讀取 lock 信息而導致靜默 region 被喚醒,從而導致 raftstore/TiKV CPU 出現抖動。具體影響如下:

  1. 不 hibernate 但 GC。一個 tikv 維護 1w 個 region 需要消耗 15% cpu 的開銷。
  2. 不 hibernate 不 GC。一個 tikv 維護 1w 個 region 需要消耗 10% cpu 的開銷。
  3. hibernate 不 GC。一個 tikv 維護 1w 個 region 需要消耗 1% cpu 的開銷。
  4. hibernate 定期 GC。一個 tikv 維護 1w 個 region 需要消耗 1% cpu 的開銷外加 GC 期間達到 10-15% 的開銷。

所以 4 相比 1 於就會有類似的尖刺現象,因爲平常的 baseline 更低

Step 3 : Delete Ranges

在上一步驟中,我們已經將鎖清理完畢,也就是所有 gc safepoint 之前的事務狀態已經明確。所以,從這一步開始,我們可以真正地開始清理數據了。在正常情況下,因爲我們底層用的是 rocksdb, 在我們明確一個版本可以清理以後,會直接調用 rocksdb 的 delete 接口去清理該 key. 但我們知道,rocksdb 本身使用的是 LSM 架構,也就是說它也有 mvcc, 它的一次刪除,最終也是轉化成了一次寫入。那數據什麼時候會真正清理呢?就需要等我們 rocksdb 的。compaction 操作來清理了,這個過程就十分漫長了,後續我們還會繼續談論這個話題。

現在從 TiDB 視角看,有一些數據清理操作,如:

  • Drop table/index
  • Truncate table
  • Drop partition
  • ...

以上操作在 GC 時需要將連續的大片數據進行刪除。對於這部分數據,我們認爲在過了 GC safepoint 之後,可以直接清理,不需要跟普通的 GC 一樣對歷史版本一邊讀一邊刪除的過程。Delete Ranges 就是我們針對這種連續的大塊刪除的優化。在這一步驟中,我們會直接對數據進行物理回收,空間會立刻被釋放出來,極大地減少這一塊 GC 對系統讀寫的壓力。

具體步驟如下:

  • 執行 SQL 階段:這些符合 delete-range 要求的 SQL 會在執行 時,記錄在系統表 mysql.gc_delete_range,並標記其刪除的時間 ts:
  • GC 階段:根據上表中刪除時間 ts 與當前 gc safepoint 進行比對,對於符合刪除條件的數據,調用 tikv 的 unsafeDestroyRange 接口對所有 tikv 發送。TiKV 在收到這個請求後,會直接繞過 raft 對本地的 rocksdb 進行 destroyRange 操作。同時,做完以後,我們會將結果記錄在系統表 gc_delete_range_done 裏面:

常見問題

這一步也很少出問題,但是不排除短期內需要刪除大量數據時,這一步執行比較慢而導致 GC 看似被卡住的情況。對於這種情況不需要做太多介入,耐心等待其完成即可。

Step 4 : Sync gc safepoint

最後,我們會將 GC safepoint 存儲到 PD 上,以通知 tikv 進行 GC。TiKV 定期會從 PD 上拿 gc safepoint, 如果發生了變更,則會拿當前的 gc safepoint 開始 tikv 本地的 GC 工作。

相關監控

我們可以從 grafana 上的 tikv-details->GC->TiKV auto GC safepoint 觀察 gc safepoint 推進是否符合預期:

常見問題

PD 一共提供了兩個接口來存 gc_safepoint, 分別爲:

  • UpdateServiceGCSafepoint:在 計算 GC safepoint 那一步調用,這步調用成功,意味着TiDB側 GC 正式開始
  • UpdateGCSafepoint 在最後這一步“ save gc safepoint toPD ”執行,這一步調用成功,意味着TiDB側 GC 正式結束。

一般情況下這兩個接口的 safepoint 應該是保持一致的,當不一致時,一般是

updateGCSafepoint< UpdateServiceGCSafepoint

也就是 GC 在 TiDB 這一側某一步卡住了。生產情況下有些卡住是符合預期的,如下面這種情況:

  • 在工具卡住 gc safepoint 之後,兩接口的 gc safepoint 出現不一致。如果此時又正好調大了 gc lifetime, 那麼在下一次 gc 成功執行完成之前,這兩個接口對應的值會一直不一致。

具體影響:無。

發表評論
所有評論
還沒有人評論,想成為第一個評論的人麼? 請在上方評論欄輸入並且點擊發布.
相關文章