無鎖併發快照在Flink MySQL CDC 2.0中的實現

前言

本文是兩個月前參加黑馬比賽時寫了一半的,爲了避免爛尾,今天補全發出來。

Flink CDC經過長時間的發展,目前無疑是實現數據源端Pipeline的最簡單直接的方式。而在1.0時代,因爲存在一些主要的缺點,它還不能稱得上是production-ready,這些缺點有:

  • Debezium底層需要通過全局鎖來保證全量和增量數據的一致性,對線上業務影響大;
  • 全量快照階段只能單線程讀取,大表同步非常耗時;
  • 全量快照階段不支持Checkpoint,一旦作業失敗只能重新同步,穩定性不佳。

但是隨着2.0版本的發佈,上述問題都得到了解決,解決方案的核心就是Netflix通過DBLog CDC框架提出的無鎖快照算法,相關論文見<<DBLog: A Watermark Based Change-Data-Capture Framework>>。本文簡單講述一下論文的思路,並對照看一下Flink CDC的實現方法。

Watermark-based Chunk Selection

爲了使得全量快照、增量讀取都實現無鎖,DBLog發明了一種名爲Watermark-based Chunk Selection的機制。名詞解釋:

  • Chunk是將源表按主鍵(必須有物理主鍵)切分成的小塊。也就是說,如果主鍵的上下界是maxmin,一個Chunk有size行,那麼最終會有(max - min) / size個Chunks。
  • Watermark的含義與Flink類似,即在Binlog Stream中插入的標記,將其分割爲包含有限量Binlog的有限集。在DBLog中,Watermark需要單獨維護在源庫中一個單行單列的表內。

下面通過引用論文中的圖示介紹一下Watermark-based Chunk Selection的流程。

(1 ~ 2) 暫停Binlog處理,並更新一個低水位Watermark,記爲L

(3) 獲取下一個Chunk並存儲在內存中,我們可以認爲該Chunk就是執行查詢當時的部分快照,當前Chunk的元信息額外借助ZK存儲。圖示的Chunk包含k1k6這些主鍵;

(4) 更新一個高水位Watermark,記爲H。由於Watermark也是一張表,所以它們會自動進入Binlog Stream,圈出一個與Chunk時間對應的Window [L, H]

(5) 恢復binlog,開始處理Chunk與Window;

(6) 從L開始讀取Window數據,並將兩者中key重合的數據從Chunk移除(因爲通過重放流就包含了同時間的快照中對應key的數據)。示例中k1k3都是重合的,所以Chunk最終只會剩下k2, k4, k5, k6

(7) 讀到H後,將Chunk中剩餘的數據插入H之前,Window數據之後。這樣就將[L, H]區間內的流式變化和批式快照整合了起來,論文中將此操作稱爲“交錯”(interleave)。

重複上述流程可以發現,隨着Chunk逐漸被讀取,同時產生的Binlog也會被增量地的處理,形成一個既包含歷史數據,也包含增量數據的統一Changelog,並且能夠保證相同主鍵的變化數據不重、不漏、不亂序,工程性極佳,且非常精妙。

Flink CDC 2.0基於FLIP-27的實現

我們知道,出於統一數據源規範、統一線程模型、支持流批一體等等考量,Flink社區早已啓用了FLIP-27 Source API作爲數據源的標準實現。FLIP-27 Source的原理可以用下圖表示,三要素分別爲:SplitEnumerator、Split、SourceReader,具體可以參考設計文檔。

結合上文所述可以發現,Split正好能夠與算法中的Chunk、Window概念對齊。以MySQL CDC爲例,MySqlSplit分爲兩種,分別是MySqlSnapshotSplit(代表Chunk)和MySqlBinlogSplit(代表Binlog Window以及純增量階段的Binlog Stream)。

無鎖快照

MySqlSourceEnumerator通過調用ChunkSplitter來分割源表,產生MySqlSplit並通過MySqlSplitAssigner分配給各個MySqlSourceReader。這部分邏輯不算難,在此不再贅述。

而負責讀取Chunk的MySqlSnapshotSplitReadTask任務中,可以明顯看出上述步驟1~4:

    @Override
    protected SnapshotResult<MySqlOffsetContext> doExecute(
            ChangeEventSourceContext context,
            MySqlOffsetContext previousOffset,
            SnapshotContext<MySqlOffsetContext> snapshotContext,
            SnapshottingTask snapshottingTask)
            throws Exception {
        // ...

        final BinlogOffset lowWatermark = currentBinlogOffset(jdbcConnection);
        LOG.info(
                "Snapshot step 1 - Determining low watermark {} for split {}",
                lowWatermark,
                snapshotSplit);
        ((SnapshotSplitReader.SnapshotSplitChangeEventSourceContextImpl) (context))
                .setLowWatermark(lowWatermark);
        signalEventDispatcher.dispatchWatermarkEvent(
                snapshotSplit, lowWatermark, SignalEventDispatcher.WatermarkKind.LOW);

        LOG.info("Snapshot step 2 - Snapshotting data");
        createDataEvents(ctx, snapshotSplit.getTableId());

        final BinlogOffset highWatermark = currentBinlogOffset(jdbcConnection);
        LOG.info(
                "Snapshot step 3 - Determining high watermark {} for split {}",
                highWatermark,
                snapshotSplit);
        signalEventDispatcher.dispatchWatermarkEvent(
                snapshotSplit, highWatermark, SignalEventDispatcher.WatermarkKind.HIGH);
        ((SnapshotSplitReader.SnapshotSplitChangeEventSourceContextImpl) (context))
                .setHighWatermark(highWatermark);

        return SnapshotResult.completed(ctx.offset);
    }

可見這裏採用的低水位與高水位值就是當時的Binlog位點值(通過執行SHOW MASTER STATUS語句獲得),避免了單獨建表保存Watermark。

負責讀取Binlog的MySqlBinlogSplitReadTask中,會先處理Binlog事件,再檢查是否達到了高水位(僅在全量同步階段會檢查,增量階段會一直同步下去)。如果已經達到,則停止繼續讀入Binlog,並向下遊發射一個BINLOG_END標記。

// 關於具體處理Binlog Event的邏輯,可參見父類`MySqlStreamingChangeEventSource#handleEvent()`
    @Override
    protected void handleEvent(MySqlOffsetContext offsetContext, Event event) {
        super.handleEvent(offsetContext, event);
        // check do we need to stop for read binlog for snapshot split.
        if (isBoundedRead()) {
            final BinlogOffset currentBinlogOffset = getBinlogPosition(offsetContext.getOffset());
            // reach the high watermark, the binlog reader should finished
            if (currentBinlogOffset.isAtOrAfter(binlogSplit.getEndingOffset())) {
                // send binlog end event
                try {
                    signalEventDispatcher.dispatchWatermarkEvent(
                            binlogSplit,
                            currentBinlogOffset,
                            SignalEventDispatcher.WatermarkKind.BINLOG_END);
                } catch (InterruptedException e) {
                    LOG.error("Send signal event error.", e);
                    errorHandler.setProducerThrowable(
                            new DebeziumException("Error processing binlog signal event", e));
                }
                // tell reader the binlog task finished
                ((SnapshotBinlogSplitChangeEventSourceContextImpl) context).finished();
            }
        }
    }

此時形成的數據佈局是:

Low Watermark | Snapshot Events | High Watermark | Binlog Events | Binlog End

接下來的整理操作(即第6、7兩步)則位於SourceRecordUtils#normalizedSplitRecords()方法中。

// 來自JdbcSourceScanFetcher#pollSplitRecords()
    public static List<SourceRecord> normalizedSplitRecords(
            SnapshotSplit snapshotSplit,
            List<SourceRecord> sourceRecords,
            SchemaNameAdjuster nameAdjuster) {
        List<SourceRecord> normalizedRecords = new ArrayList<>();
        Map<Struct, SourceRecord> snapshotRecords = new HashMap<>();
        List<SourceRecord> binlogRecords = new ArrayList<>();
        if (!sourceRecords.isEmpty()) {

            SourceRecord lowWatermark = sourceRecords.get(0);
            checkState(
                    isLowWatermarkEvent(lowWatermark),
                    String.format(
                            "The first record should be low watermark signal event, but is %s",
                            lowWatermark));
            SourceRecord highWatermark = null;
            int i = 1;
            for (; i < sourceRecords.size(); i++) {
                SourceRecord sourceRecord = sourceRecords.get(i);
                if (!isHighWatermarkEvent(sourceRecord)) {
                    snapshotRecords.put((Struct) sourceRecord.key(), sourceRecord);
                } else {
                    highWatermark = sourceRecord;
                    i++;
                    break;
                }
            }

            if (i < sourceRecords.size() - 1) {
                List<SourceRecord> allBinlogRecords =
                        sourceRecords.subList(i, sourceRecords.size() - 1);
                for (SourceRecord binlog : allBinlogRecords) {
                    if (isDataChangeRecord(binlog)) {
                        Object[] key =
                                getSplitKey(snapshotSplit.getSplitKeyType(), binlog, nameAdjuster);
                        if (splitKeyRangeContains(
                                key, snapshotSplit.getSplitStart(), snapshotSplit.getSplitEnd())) {
                            binlogRecords.add(binlog);
                        }
                    }
                }
            }
            checkState(
                    isHighWatermarkEvent(highWatermark),
                    String.format(
                            "The last record should be high watermark signal event, but is %s",
                            highWatermark));
            normalizedRecords =
                    upsertBinlog(lowWatermark, highWatermark, snapshotRecords, binlogRecords);
        }
        return normalizedRecords;
    }

該方法的操作步驟是:

  1. 用Watermark將所有數據分爲快照(snapshotRecords)和Binlog(binlogRecords)兩部分,注意處理Binlog時需要做過濾,只保留屬於自己Split的那部分日誌;
  2. 調用upsertBinlog()方法將binlogRecords以UPSERT和DELETE語義覆蓋到snapshotRecords中,返回的就是interleave好的Changelog數據了,佈局如下。
Low Watermark | Normalized Events | High Watermark

併發一致性

原生的無鎖快照算法仍然是單線程執行的,不過在FLIP-27的模型下,多個SourceReader就可以自然並行地獲取Chunk及讀取Binlog。由於Chunk的key之間沒有交集,所以即使它們處理的Binlog Window之間有交集(這是必然會發生的),也仍然可以保證數據一致性。那麼,如何在全量階段併發完成並轉向增量同步時也保證數據絕對正確呢?

在一個Snapshot Split處理完成時,MysqlSourceReader會將其放入finishedUnackedSplits集合,並觸發完成事件彙報給MySqlSourceEnumerator。該完成事件中包含Split的ID以及對應的高水位值。

// 由MySqlSourceReader#onSplitFinished()調用
    private void reportFinishedSnapshotSplitsIfNeed() {
        if (!finishedUnackedSplits.isEmpty()) {
            final Map<String, BinlogOffset> finishedOffsets = new HashMap<>();
            for (MySqlSnapshotSplit split : finishedUnackedSplits.values()) {
                finishedOffsets.put(split.splitId(), split.getHighWatermark());
            }
            FinishedSnapshotSplitsReportEvent reportEvent =
                    new FinishedSnapshotSplitsReportEvent(finishedOffsets);
            context.sendSourceEventToCoordinator(reportEvent);
            LOG.debug(
                    "The subtask {} reports offsets of finished snapshot splits {}.",
                    subtaskId,
                    finishedOffsets);
        }
    }

而產生增量階段所需的long-running Binlog Split之前,則會從所有高水位值中選擇最小的那一個作爲起始的Binlog Offset。只要該位點對應的Binlog沒有被purge掉,就一定不會漏掉任何數據了。

// MySqlHybridSplitAssigner#createBinlogSplit()
    private MySqlBinlogSplit createBinlogSplit() {
        // ...
        BinlogOffset minBinlogOffset = null;
        for (MySqlSnapshotSplit split : assignedSnapshotSplit) {
            // find the min binlog offset
            BinlogOffset binlogOffset = splitFinishedOffsets.get(split.splitId());
            if (minBinlogOffset == null || binlogOffset.isBefore(minBinlogOffset)) {
                minBinlogOffset = binlogOffset;
            }
            finishedSnapshotSplitInfos.add(
                    new FinishedSnapshotSplitInfo(
                            split.getTableId(),
                            split.splitId(),
                            split.getSplitStart(),
                            split.getSplitEnd(),
                            binlogOffset));
        }
        // ...
    }

全量階段的Checkpoint支持

容易推斷,在FLIP-27的體系下,如果我們對Split的分配狀態和Split本身的狀態做快照,那麼即使CDC的全量階段失敗,也可以從上述狀態恢復現場,進行斷點續傳。例如,對於等待分配的Snapshot Split集合,在Checkpoint時需要記錄如下信息:

    public SnapshotPendingSplitsState(
            List<TableId> alreadyProcessedTables,
            List<MySqlSnapshotSplit> remainingSplits,
            Map<String, MySqlSnapshotSplit> assignedSplits,
            Map<String, BinlogOffset> splitFinishedOffsets,
            AssignerStatus assignerStatus,
            List<TableId> remainingTables,
            boolean isTableIdCaseSensitive,
            boolean isRemainingTablesCheckpointed) {
        // ...
}

又例如對於一個Binlog Split,在Checkpoint時需要記錄如下信息:

    @Nullable private BinlogOffset startingOffset;
    @Nullable private BinlogOffset endingOffset;
    private final Map<TableId, TableChange> tableSchemas;

    public MySqlBinlogSplitState(MySqlBinlogSplit split) {
        super(split);
        this.startingOffset = split.getStartingOffset();
        this.endingOffset = split.getEndingOffset();
        this.tableSchemas = split.getTableSchemas();
    }

具體源碼就不再分析了,看官可自行參考。

The End

晚安晚安~

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