变更数据捕获 (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
);Copy
目前,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
只有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"
}Copy
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"
}Copy
数据说明:为了减少传输带宽,如果是批量D或者I的时候,在一条数据中传输多个相同类型的数据。比如说上述I操作,columns表示字段名称,在rows中定义多条插入数据。由于U被拆分为,D和I,数据传输为单条数据传输,也就是rows中不会出现多条数据的情况,并且确保顺序写入kafka同一分区。
下图描述了Maxwell CDC format相关流程:
从上图我们可以知道,Flink 会通过 SPI 机制将 classpath 下注册的所有工厂类加载进来,包括 DynamicTableFactory、DeserializationFormatFactory 等等。同时通过DDL中的format的值与DeserializationFormatFactory工厂类的factoryIdentifier()返回值进行匹配,从而确定使用哪个工厂类。
然后再通过 DeserializationFormatFactory 中的 createDecodingFormat(…) 方法,将反序列化对象提供给 DynamicTableSource。
所以说,上述流程主要完成了以下工作:
根据上述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;
}
}Copy
定义反序列化器:
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();
}
}Copy
具体来看一下上述代码,工厂类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();
}Copy
再来看一下核心方法,deserialize(byte[] bytes, Collector\<RowData> 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://developer.aliyun.com/article/771438
本文为从大数据到人工智能博主「xiaozhch5」的原创文章,遵循CC 4.0 BY-SA版权协议,转载请附上原文出处链接及本声明。