Skip to content

Commit

Permalink
[cdc] refactor mysql cdc action uniformly uses flatmap to process rec…
Browse files Browse the repository at this point in the history
…ords (#2220)
  • Loading branch information
zhuangchong authored Nov 6, 2023
1 parent c211c75 commit f2c152e
Show file tree
Hide file tree
Showing 12 changed files with 171 additions and 308 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,8 @@ public void build() throws Exception {
source,
WatermarkStrategy.noWatermarks(),
sourceName())
.flatMap(recordParser))
.flatMap(recordParser)
.name("Parse"))
.withParserFactory(parserFactory)
.withTable(fileStoreTable)
.withIdentifier(identifier)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,8 @@ public void build() throws Exception {
new FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord>()
.withInput(
env.fromSource(source, WatermarkStrategy.noWatermarks(), "MongoDB Source")
.flatMap(new MongoDBRecordParser(caseSensitive, mongodbConfig)))
.flatMap(new MongoDBRecordParser(caseSensitive, mongodbConfig))
.name("Parse"))
.withParserFactory(parserFactory)
.withCatalogLoader(catalogLoader())
.withDatabase(database)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,8 @@ public void build() throws Exception {
new MongoDBRecordParser(
caseSensitive,
computedColumns,
mongodbConfig)))
mongodbConfig))
.name("Parse"))
.withParserFactory(parserFactory)
.withTable(fileStoreTable)
.withIdentifier(identifier)
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,9 @@
import org.apache.paimon.flink.action.cdc.mysql.schema.MySqlTableInfo;
import org.apache.paimon.flink.sink.cdc.EventParser;
import org.apache.paimon.flink.sink.cdc.FlinkCdcSyncDatabaseSinkBuilder;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordSchemaBuilder;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.FileStoreTable;
Expand All @@ -46,7 +49,6 @@

import javax.annotation.Nullable;

import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -286,28 +288,27 @@ public void build() throws Exception {
monitoredTables,
excludedTables));

String serverTimeZone = mySqlConfig.get(MySqlSourceOptions.SERVER_TIME_ZONE);
ZoneId zoneId = serverTimeZone == null ? ZoneId.systemDefault() : ZoneId.of(serverTimeZone);
TypeMapping typeMapping = this.typeMapping;
MySqlTableSchemaBuilder schemaBuilder =
new MySqlTableSchemaBuilder(tableConfig, caseSensitive, typeMapping);
RichCdcMultiplexRecordSchemaBuilder schemaBuilder =
new RichCdcMultiplexRecordSchemaBuilder(tableConfig, caseSensitive);

EventParser.Factory<String> parserFactory =
TypeMapping typeMapping = this.typeMapping;
MySqlRecordParser recordParser =
new MySqlRecordParser(mySqlConfig, caseSensitive, typeMapping, metadataConverters);
EventParser.Factory<RichCdcMultiplexRecord> parserFactory =
() ->
new MySqlDebeziumJsonEventParser(
zoneId,
caseSensitive,
tableNameConverter,
new RichCdcMultiplexRecordEventParser(
schemaBuilder,
includingPattern,
excludingPattern,
typeMapping,
metadataConverters);
tableNameConverter);

String database = this.database;
MultiTablesSinkMode mode = this.mode;
new FlinkCdcSyncDatabaseSinkBuilder<String>()
.withInput(env.fromSource(source, WatermarkStrategy.noWatermarks(), "MySQL Source"))
new FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord>()
.withInput(
env.fromSource(source, WatermarkStrategy.noWatermarks(), "MySQL Source")
.flatMap(recordParser)
.name("Parse"))
.withParserFactory(parserFactory)
.withDatabase(database)
.withCatalogLoader(catalogLoader())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
import org.apache.paimon.flink.action.cdc.mysql.schema.MySqlTableInfo;
import org.apache.paimon.flink.sink.cdc.CdcSinkBuilder;
import org.apache.paimon.flink.sink.cdc.EventParser;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.table.FileStoreTable;

Expand All @@ -40,7 +42,6 @@
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.configuration.Configuration;

import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
Expand Down Expand Up @@ -204,7 +205,7 @@ public void build() throws Exception {
try {
fileStoreTable = (FileStoreTable) catalog.getTable(identifier);
fileStoreTable = fileStoreTable.copy(tableConfig);
if (computedColumns.size() > 0) {
if (!computedColumns.isEmpty()) {
List<String> computedFields =
computedColumns.stream()
.map(ComputedColumn::columnName)
Expand All @@ -229,23 +230,27 @@ public void build() throws Exception {
.collect(Collectors.joining("|"));
MySqlSource<String> source = MySqlActionUtils.buildMySqlSource(mySqlConfig, tableList);

String serverTimeZone = mySqlConfig.get(MySqlSourceOptions.SERVER_TIME_ZONE);
ZoneId zoneId = serverTimeZone == null ? ZoneId.systemDefault() : ZoneId.of(serverTimeZone);
TypeMapping typeMapping = this.typeMapping;
EventParser.Factory<String> parserFactory =
() ->
new MySqlDebeziumJsonEventParser(
zoneId,
caseSensitive,
computedColumns,
typeMapping,
metadataConverters);

CdcSinkBuilder<String> sinkBuilder =
new CdcSinkBuilder<String>()
MySqlRecordParser recordParser =
new MySqlRecordParser(
mySqlConfig,
caseSensitive,
computedColumns,
typeMapping,
metadataConverters);

EventParser.Factory<RichCdcMultiplexRecord> parserFactory =
() -> new RichCdcMultiplexRecordEventParser(caseSensitive);

CdcSinkBuilder<RichCdcMultiplexRecord> sinkBuilder =
new CdcSinkBuilder<RichCdcMultiplexRecord>()
.withInput(
env.fromSource(
source, WatermarkStrategy.noWatermarks(), "MySQL Source"))
source,
WatermarkStrategy.noWatermarks(),
"MySQL Source")
.flatMap(recordParser)
.name("Parse"))
.withParserFactory(parserFactory)
.withTable(fileStoreTable)
.withIdentifier(identifier)
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -95,8 +95,7 @@ public void processElement(T raw, Context context, Collector<Void> collector) th
parser.parseNewTable()
.ifPresent(
schema -> {
Identifier identifier =
new Identifier(database, parser.parseTableName());
Identifier identifier = new Identifier(database, tableName);
try {
catalog.createTable(identifier, schema, true);
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ public void processElement(T raw, Context context, Collector<Void> collector) th
parser.setRawEvent(raw);
String tableName = parser.parseTableName();
List<DataField> schemaChange = parser.parseSchemaChange();
if (schemaChange.size() > 0) {
if (!schemaChange.isEmpty()) {
context.output(getUpdatedDataFieldsOutputTag(tableName), schemaChange);
}
parser.parseRecords()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public void processElement(T raw, Context context, Collector<CdcRecord> collecto
throws Exception {
parser.setRawEvent(raw);
List<DataField> schemaChange = parser.parseSchemaChange();
if (schemaChange.size() > 0) {
if (!schemaChange.isEmpty()) {
context.output(NEW_DATA_FIELD_LIST_OUTPUT_TAG, schemaChange);
}
parser.parseRecords().forEach(collector::collect);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,10 @@ public RichCdcMultiplexRecord(
this.cdcRecord = cdcRecord;
}

public String databaseName() {
return databaseName;
}

public String tableName() {
return tableName;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.flink.sink.cdc;

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.action.cdc.TableNameConverter;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.types.DataField;
Expand Down Expand Up @@ -84,7 +85,7 @@ public void setRawEvent(RichCdcMultiplexRecord record) {

@Override
public String parseTableName() {
return tableNameConverter.convert(currentTable);
return tableNameConverter.convert(Identifier.create(record.databaseName(), currentTable));
}

@Override
Expand Down Expand Up @@ -139,6 +140,8 @@ private boolean shouldSynchronizeCurrentTable() {
}

private boolean shouldCreateCurrentTable() {
return shouldSynchronizeCurrentTable && createdTables.add(currentTable);
return shouldSynchronizeCurrentTable
&& !record.fieldTypes().isEmpty()
&& createdTables.add(parseTableName());
}
}

0 comments on commit f2c152e

Please sign in to comment.