Flink cdc自定义format格式数据源

本文涉及的产品
实时计算 Flink 版,5000CU*H 3个月
简介: 变更数据捕获 (CDC) 已成为一种流行的模式,本文介绍如何通过自定义format来获取不同格式cdc数据源

总览

变更数据捕获 (CDC) 已成为一种流行的模式,用于从数据库捕获已提交的变更并将这些变更传播给下游消费者,例如保持多个数据存储同步并避免常见的陷阱,例如双重写入。 能够轻松地将这些变更日志摄取和解释到 Table API/SQL 中一直是 Flink 社区的一个非常需要的功能,现在 Flink 1.11 可以实现。

为了将 Table API/SQL 的范围扩展到 CDC 等用例,Flink 1.11 引入了具有变更日志模式的新表源和接收器接口(请参阅新的 TableSource 和 TableSink 接口)并支持 Debezium 和 Canal 格式(FLIP-105)。 这意味着动态表源不再仅限于追加操作,并且可以摄取这些外部更改日志(插入事件),将它们解释为更改操作(插入、更新、删除事件)并使用更改类型将它们发送到下游。

用户必须在其 CREATE TABLE 语句中指定“format=debezium-json”或“format=canal-json”才能使用 SQL DDL 使用更改日志。

例如:

CREATE TABLE my_table (
  ...
) WITH (
  'connector'='...', -- e.g. 'kafka'
  'format'='debezium-json',
  'debezium-json.schema-include'='true' -- default: false (Debezium can be configured to include or exclude the message schema)
  'debezium-json.ignore-parse-errors'='true' -- default: false
);

目前,Flink内置Debezium CDC、Canal CDC、Maxwell CDC三种format并以kafka和Filesystem作为connector。但是,我们在使用的时候发现,其实上述三种CDC format是远远不能满足我们的需求的公司客户有各种各样的自定义CDC格式。下面列举其中一种格式,并针对此格式自定义CDC format。

有什么参考资料?

参考官方CDC format用例,具体为flink-json-1.1x.x.jar包中提供的内容。

org.apache.flink.formats.json.canal

org.apache.flink.formats.json.debezium

org.apache.flink.formats.json.maxwell

本文自定义CDC数据源格式

只有I和D格式,U已经被拆解为D和I

I格式数据:

{
  "columns": [
    "col1",
    "col2",
    "col3",
    "col4",
    "col5"
  ],
  "rows": [
    {
      "after": [
        "a1",
        "b1",
        "c1",
        "d1",
        "e1"
      ],
      "current_ts": "2021-09-23 17:11:33",
      "op_ts": "2021-09-23 17:11:33",
      "op_type": "I",
      "pos": "09231632388293978189"
    },
    {
      "after": [
        "a2",
        "b2",
        "c2",
        "d2",
        "e2"
      ],
      "current_ts": "2021-09-23 17:11:33",
      "op_ts": "2021-09-23 17:11:33",
      "op_type": "I",
      "pos": "09231632388293978189"
    }
  ],
  "table": "table1"
}

D格式数据

{
  "columns": [
    "col1",
    "col2",
    "col3",
    "col4",
    "col5"
  ],
  "rows": [
    {
      "before": [
        "a1",
        "b1",
        "c1",
        "d1",
        "e1"
      ],
      "current_ts": "2021-09-23 17:11:33",
      "op_ts": "2021-09-23 17:11:33",
      "op_type": "D",
      "pos": "09231632388293978189"
    },
    {
      "before": [
        "a2",
        "b2",
        "c2",
        "d2",
        "e2"
      ],
      "current_ts": "2021-09-23 17:11:33",
      "op_ts": "2021-09-23 17:11:33",
      "op_type": "D",
      "pos": "09231632388293978189"
    }
  ],
  "table": "table1"
}

数据说明:为了减少传输带宽,如果是批量D或者I的时候,在一条数据中传输多个相同类型的数据。比如说上述I操作,columns表示字段名称,在rows中定义多条插入数据。由于U被拆分为,D和I,数据传输为单条数据传输,也就是rows中不会出现多条数据的情况,并且确保顺序写入kafka同一分区。

从何写起

下图描述了Maxwell CDC format相关流程:

[外链图片转存失败,源站可能有防盗链机制,建议将图片保存下来直接上传(img-g4r7l3XK-1634882436700)(https://obs-githubhelper.obs.cn-east-3.myhuaweicloud.com/blog-images/category/bigdata/flink/flink-cdc-user-define-format/image%20(1)].png)

从上图我们可以知道,Flink 会通过 SPI 机制将 classpath 下注册的所有工厂类加载进来,包括 DynamicTableFactory、DeserializationFormatFactory 等等。同时通过DDL中的format的值与DeserializationFormatFactory工厂类的factoryIdentifier()返回值进行匹配,从而确定使用哪个工厂类。

然后再通过 DeserializationFormatFactory 中的 createDecodingFormat(...) 方法,将反序列化对象提供给 DynamicTableSource。

所以说,上述流程主要完成了以下工作:

  • 定义工厂类(DeserializationFormatFactory),即 MaxwellJsonFormatFactory,负责编译时根据 ‘format’ = ‘maxwell-json’创建对应的反序列化器
  • 定义反序列化类(DeserializationSchema),即MaxwellJsonDeserializationSchema,负责运行时的解析,根据固定格式将 CDC 数据转换成 Flink 系统能认识的 INSERT/DELETE/UPDATE 消息,如 RowData。
  • Service 注册文件:需要添加 Service 文件 META-INF/services/org.apache.flink.table.factories.Factory ,并在其中增加一行上述实现的 MaxwellJsonFormatFactory 类路径。

开始定义你的cdc format

根据上述Maxwell CDC format相关流程,我们先定义工厂类,具体如下:

import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonOptions;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DeserializationFormatFactory;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.factories.SerializationFormatFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;

import java.util.Collections;
import java.util.HashSet;
import java.util.Set;

public class AnalysisJsonFormatFactory  implements DeserializationFormatFactory, SerializationFormatFactory {

    public static final String IDENTIFIER = "analysis-json";

    public AnalysisJsonFormatFactory() {
    }

    @Override
    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(DynamicTableFactory.Context context, ReadableConfig formatOptions) {
        FactoryUtil.validateFactoryOptions(this, formatOptions);
        AnalysisJsonOptions.validateDecodingFormatOptions(formatOptions);
        boolean ignoreParseErrors = (Boolean)formatOptions.get(AnalysisJsonOptions.IGNORE_PARSE_ERRORS);
        TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions);
        return new DecodingFormat<DeserializationSchema<RowData>>() {
            @Override
            public DeserializationSchema<RowData> createRuntimeDecoder(DynamicTableSource.Context context, DataType dataType) {
                // 表的字段名和数据类型
                RowType rowType = (RowType) dataType.getLogicalType();
                TypeInformation<RowData> rowDataTypeInformation = context.createTypeInformation(dataType);
                return new AnalysisJsonDeserializationSchema(rowType, rowDataTypeInformation, ignoreParseErrors, timestampFormat);
            }

            @Override
            public ChangelogMode getChangelogMode() {
                return ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT).addContainedKind(RowKind.DELETE).build();
            }
        };
    }

    @Override
    public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(DynamicTableFactory.Context context, ReadableConfig readableConfig) {

        return null;
    }

    @Override
    public String factoryIdentifier() {
        return "analysis-json";
    }

    @Override
    public Set<ConfigOption<?>> requiredOptions() {
        return Collections.emptySet();
    }

    @Override
    public Set<ConfigOption<?>> optionalOptions() {
        Set<ConfigOption<?>> options = new HashSet<>();
        options.add(AnalysisJsonOptions.IGNORE_PARSE_ERRORS);
        return options;
    }
}

定义反序列化器:

import com.alibaba.fastjson.JSONArray;
import com.alibaba.fastjson.JSONObject;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;
import org.apache.flink.util.Collector;

import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;

public class AnalysisJsonDeserializationSchema  implements DeserializationSchema<RowData> {

    private static final long serialVersionUID = 1L;
    private static final String OP_INSERT = "I";
    private static final String OP_DELETE = "D";
    private final JsonRowDataDeserializationSchema jsonDeserializer;
    private final TypeInformation<RowData> resultTypeInfo;
    private final boolean ignoreParseErrors;
    private final int fieldCount;


    public AnalysisJsonDeserializationSchema(RowType rowType, TypeInformation<RowData> resultTypeInfo, boolean ignoreParseErrors, TimestampFormat timestampFormatOption) {
        this.resultTypeInfo = resultTypeInfo;
        this.ignoreParseErrors = ignoreParseErrors;
        this.fieldCount = rowType.getFieldCount();
        this.jsonDeserializer = new JsonRowDataDeserializationSchema(this.createJsonRowType(), resultTypeInfo, false, ignoreParseErrors, timestampFormatOption);
    }

    @Override
    public RowData deserialize(byte[] bytes) throws IOException {
        throw new RuntimeException("Please invoke DeserializationSchema#deserialize(byte[], Collector<RowData>) instead.");
    }

    public void deserialize(byte[] bytes, Collector<RowData> out) throws IOException {
        if (bytes != null && bytes.length != 0) {
            RowData rowData = this.jsonDeserializer.deserialize(bytes);
            String columns = rowData.getString(0).toString();
            JSONArray columnsJSONArray = JSONArray.parseArray(columns);

            String rows = rowData.getString(1).toString();
            JSONArray rowsJSONArray = JSONArray.parseArray(rows);
            for (Object object : rowsJSONArray) {
                JSONObject rowJSONObject = (JSONObject) object;
                Map<String, String> outputData = new HashMap<>();
                if (OP_INSERT.equals(rowJSONObject.getString("op_type"))) {

                    JSONArray after = rowJSONObject.getJSONArray("after");
                    int index = 0;
                    for (Object column : columnsJSONArray) {
                        outputData.put(column.toString(), after.getString(index));
                        index++;
                    }
                    GenericRowData insert = new GenericRowData(1);
                    insert.setField(0, StringData.fromBytes(JSONObject.toJSONBytes(outputData)));
                    insert.setRowKind(RowKind.INSERT);
                    out.collect(insert);
                }
                else {
                    JSONArray before = rowJSONObject.getJSONArray("before");
                    int index = 0;
                    for (Object column : columnsJSONArray) {
                        outputData.put(column.toString(), before.getString(index));
                        index++;
                    }
                    GenericRowData delete = new GenericRowData(1);
                    delete.setField(0, StringData.fromBytes(JSONObject.toJSONBytes(outputData)));
                    delete.setRowKind(RowKind.DELETE);
                    out.collect(delete);
                }
            }
        }
    }

    @Override
    public boolean isEndOfStream(RowData rowData) {
        return false;
    }

    @Override
    public TypeInformation<RowData> getProducedType() {
        return this.resultTypeInfo;
    }

    public boolean equals(Object o) {
        if (this == o) {
            return true;
        } else if (o != null && this.getClass() == o.getClass()) {
            AnalysisJsonDeserializationSchema that = (AnalysisJsonDeserializationSchema)o;
            return this.ignoreParseErrors == that.ignoreParseErrors && this.fieldCount == that.fieldCount && Objects.equals(this.jsonDeserializer, that.jsonDeserializer) && Objects.equals(this.resultTypeInfo, that.resultTypeInfo);
        } else {
            return false;
        }
    }

    public int hashCode() {
        return Objects.hash(this.jsonDeserializer, this.resultTypeInfo, this.ignoreParseErrors, this.fieldCount);
    }

    private RowType createJsonRowType() {
        return (RowType) DataTypes.ROW(
                DataTypes.FIELD("columns", DataTypes.STRING()),
                DataTypes.FIELD("rows", DataTypes.STRING()),
                DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType();
    }
}

具体来看一下上述代码,工厂类AnalysisJsonFormatFactory的写法相对固定,在createDecodingFormat中返回DecodingFormat,并重写createRuntimeDecoder和getChangelogMode方法,其中,createRuntimeDecoder返回的为上述定义的AnalysisJsonDeserializationSchema,getChangelogMode则返回我们数据格式中可能存在的操作形式,本位为INSERT以及DELETE。

再来看一下AnalysisJsonDeserializationSchema,其中this.jsonDeserializer则描述了如何反序列化原始kafka数据,在本例中,由于原始数据格式固定,所以直接定义其格式为:

    private RowType createJsonRowType() {
        return (RowType) DataTypes.ROW(
                DataTypes.FIELD("columns", DataTypes.STRING()),
                DataTypes.FIELD("rows", DataTypes.STRING()),
                DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType();
    }

再来看一下核心方法,deserialize(byte[] bytes, Collector out),该方法接受原始byte[]类型原始数据,并通过out返回解析完的数据。

该方法通过this.jsonDeserializer将原始数据反序列化为rowData,那么后续则可以通过此rowData获取原始数据中的columns、rows以及table中的值。

上述代码中解析rows中的数据,并判断其类型为I或者D,并封装为GenericRowData返回。

在上一个步骤中,我们说到要注册工厂类,所以我们需要在项目resources目录中新建META-INF/services/org.apache.flink.table.factories.Factory,并在org.apache.flink.table.factories.Factory文件中填入:AnalysisJsonFormatFactory工厂类的完整路径。

完整代码示例

https://github.com/xiaozhch5/flink-cdc-format.git

参考资料

https://lrting.top/special/hudi/

https://developer.aliyun.com/article/771438

https://flink.apache.org/news/2020/07/06/release-1.11.0.html#table-apisql-support-for-change-data-capture-cdc

相关实践学习
基于Hologres轻松玩转一站式实时仓库
本场景介绍如何利用阿里云MaxCompute、实时计算Flink和交互式分析服务Hologres开发离线、实时数据融合分析的数据大屏应用。
Linux入门到精通
本套课程是从入门开始的Linux学习课程,适合初学者阅读。由浅入深案例丰富,通俗易懂。主要涉及基础的系统操作以及工作中常用的各种服务软件的应用、部署和优化。即使是零基础的学员,只要能够坚持把所有章节都学完,也一定会受益匪浅。
相关文章
|
1月前
|
消息中间件 Java Kafka
Flink CDC 在外部查询某个 job 中的表数据
【2月更文挑战第27天】Flink CDC 在外部查询某个 job 中的表数据
44 5
|
1月前
|
API 数据库 流计算
有大佬知道在使用flink cdc实现数据同步,如何实现如果服务停止了对数据源表的某个数据进行删除操作,重启服务之后目标表能进行对源表删除的数据进行删除吗?
【2月更文挑战第27天】有大佬知道在使用flink cdc实现数据同步,如何实现如果服务停止了对数据源表的某个数据进行删除操作,重启服务之后目标表能进行对源表删除的数据进行删除吗?
50 3
|
1月前
|
消息中间件 Kafka 流计算
如果有多个版本的Flink CDC在同一环境中运行,可能会导致Debezium版本冲突
【2月更文挑战第30天】如果有多个版本的Flink CDC在同一环境中运行,可能会导致Debezium版本冲突
20 2
|
1月前
|
Java 关系型数据库 MySQL
Flink CDC有见这个报错不?
【2月更文挑战第29天】Flink CDC有见这个报错不?
22 2
|
2月前
|
消息中间件 Kafka Apache
Apache Flink 是一个开源的分布式流处理框架
Apache Flink 是一个开源的分布式流处理框架
482 5
|
1月前
|
SQL Java API
官宣|Apache Flink 1.19 发布公告
Apache Flink PMC(项目管理委员)很高兴地宣布发布 Apache Flink 1.19.0。
1336 1
官宣|Apache Flink 1.19 发布公告
|
1月前
|
SQL Apache 流计算
Apache Flink官方网站提供了关于如何使用Docker进行Flink CDC测试的文档
【2月更文挑战第25天】Apache Flink官方网站提供了关于如何使用Docker进行Flink CDC测试的文档
142 3
|
1月前
|
XML Java Apache
Apache Flink自定义 logback xml配置
Apache Flink自定义 logback xml配置
150 0
|
1月前
|
消息中间件 Java Kafka
Apache Hudi + Flink作业运行指南
Apache Hudi + Flink作业运行指南
85 1
|
1月前
|
缓存 分布式计算 Apache
Apache Hudi与Apache Flink更好地集成,最新方案了解下?
Apache Hudi与Apache Flink更好地集成,最新方案了解下?
60 0