Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.doris.cdcclient.source.deserialize.DeserializeResult;
import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer;
import org.apache.doris.cdcclient.utils.SchemaChangeManager;
import org.apache.doris.job.cdc.request.JobBaseConfig;
import org.apache.doris.job.cdc.request.JobBaseRecordRequest;

import org.apache.flink.cdc.connectors.base.utils.SerializerUtils;
Expand All @@ -37,6 +38,7 @@
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.document.DocumentWriter;
import io.debezium.relational.Column;
import io.debezium.relational.TableId;
import io.debezium.relational.history.TableChanges;
import lombok.Getter;
Expand All @@ -63,6 +65,49 @@ public abstract class AbstractCdcSourceReader implements SourceReader {
protected SourceRecordDeserializer<SourceRecord, DeserializeResult> serializer;
protected Map<TableId, TableChanges.TableChange> tableSchemas;

private final Map<String, Class<?>> splitKeyClassCache = new ConcurrentHashMap<>();

protected abstract Class<?> probeSplitKeyClass(
TableId tableId, Column splitColumn, JobBaseConfig jobConfig);

protected Class<?> resolveSplitKeyClass(
TableId tableId, Column splitColumn, JobBaseConfig jobConfig) {
String key = tableId.identifier() + "." + splitColumn.name();
return splitKeyClassCache.computeIfAbsent(
key, k -> probeSplitKeyClass(tableId, splitColumn, jobConfig));
}

protected static Object[] convertBounds(Object[] raw, Class<?> target, ObjectMapper mapper) {
if (raw == null) {
return null;
}
Object[] out = new Object[raw.length];
for (int i = 0; i < raw.length; i++) {
out[i] = convertBound(raw[i], target, mapper);
}
return out;
}

private static Object convertBound(Object v, Class<?> target, ObjectMapper mapper) {
if (v == null) {
return null;
}
if (target.isInstance(v)) {
return v;
}
String s = v.toString();
if (target == java.sql.Date.class) {
return java.sql.Date.valueOf(s);
}
if (target == java.sql.Timestamp.class) {
return java.sql.Timestamp.valueOf(s);
}
if (target == java.sql.Time.class) {
return java.sql.Time.valueOf(s);
}
return mapper.convertValue(v, target);
}

/**
* Load tableSchemas from a JSON string (produced by {@link #serializeTableSchemas()}). Used
* when a binlog/stream split is resumed from FE-persisted state.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -614,16 +614,25 @@ protected abstract Fetcher<SourceRecords, SourceSplitBase> getBinlogSplitReader(
createSnapshotSplit(Map<String, Object> offset, JobBaseConfig jobConfig) {
SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class);
TableId tableId = TableId.parse(snapshotSplit.getTableId(), false);
Object[] splitStart = snapshotSplit.getSplitStart();
Object[] splitEnd = snapshotSplit.getSplitEnd();
List<String> splitKeys = snapshotSplit.getSplitKey();
Map<TableId, TableChanges.TableChange> tableSchemas = getTableSchemas(jobConfig);
TableChanges.TableChange tableChange = tableSchemas.get(tableId);
Preconditions.checkNotNull(
tableChange, "Can not find table " + tableId + " in job " + jobConfig.getJobId());
// only support one split key
String splitKey = splitKeys.get(0);
io.debezium.relational.Column splitColumn = tableChange.getTable().columnWithName(splitKey);
Column splitColumn = tableChange.getTable().columnWithName(splitKey);
Preconditions.checkNotNull(
splitColumn,
"Split key column "
+ splitKey
+ " not found in table "
+ tableId
+ " for job "
+ jobConfig.getJobId());
Class<?> keyClass = resolveSplitKeyClass(tableId, splitColumn, jobConfig);
Object[] splitStart = convertBounds(snapshotSplit.getSplitStart(), keyClass, objectMapper);
Object[] splitEnd = convertBounds(snapshotSplit.getSplitEnd(), keyClass, objectMapper);
RowType splitType = getSplitType(splitColumn);
org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit split =
new org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit(
Expand Down Expand Up @@ -660,6 +669,7 @@ private Tuple2<SourceSplitBase, Boolean> createStreamSplit(
.sorted(Comparator.comparing(AbstractSourceSplit::getSplitId))
.toList();

Map<TableId, TableChanges.TableChange> tableSchemas = getTableSchemas(config);
for (SnapshotSplit split : assignedSplitLists) {
// find the min offset
Map<String, String> offsetMap = split.getHighWatermark();
Expand All @@ -670,12 +680,29 @@ private Tuple2<SourceSplitBase, Boolean> createStreamSplit(
if (maxOffsetFinishSplits == null || sourceOffset.isAfter(maxOffsetFinishSplits)) {
maxOffsetFinishSplits = sourceOffset;
}
TableId tid = TableId.parse(split.getTableId());
TableChanges.TableChange tableChange = tableSchemas.get(tid);
Preconditions.checkNotNull(
tableChange, "Can not find table " + tid + " in job " + config.getJobId());
String splitKey = split.getSplitKey().get(0);
Column splitColumn = tableChange.getTable().columnWithName(splitKey);
Preconditions.checkNotNull(
splitColumn,
"Split key column "
+ splitKey
+ " not found in table "
+ tid
+ " for job "
+ config.getJobId());
Class<?> keyClass = resolveSplitKeyClass(tid, splitColumn, config);
Object[] start = convertBounds(split.getSplitStart(), keyClass, objectMapper);
Object[] end = convertBounds(split.getSplitEnd(), keyClass, objectMapper);
finishedSnapshotSplitInfos.add(
new FinishedSnapshotSplitInfo(
TableId.parse(split.getTableId()),
tid,
split.getSplitId(),
split.getSplitStart(),
split.getSplitEnd(),
start,
end,
sourceOffset,
getOffsetFactory()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords;
import org.apache.flink.cdc.connectors.mysql.source.utils.ChunkUtils;
import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils;
import org.apache.flink.cdc.connectors.mysql.source.utils.StatementUtils;
import org.apache.flink.cdc.connectors.mysql.source.utils.TableDiscoveryUtils;
import org.apache.flink.cdc.connectors.mysql.table.StartupMode;
import org.apache.flink.cdc.connectors.mysql.table.StartupOptions;
Expand All @@ -69,7 +70,9 @@
import org.apache.kafka.connect.source.SourceRecord;

import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -590,8 +593,6 @@ private MySqlSnapshotSplit createSnapshotSplit(
Map<String, Object> offset, JobBaseConfig jobConfig) throws JsonProcessingException {
SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class);
TableId tableId = TableId.parse(snapshotSplit.getTableId());
Object[] splitStart = snapshotSplit.getSplitStart();
Object[] splitEnd = snapshotSplit.getSplitEnd();
List<String> splitKeys = snapshotSplit.getSplitKey();
Map<TableId, TableChanges.TableChange> tableSchemas = getTableSchemas(jobConfig);
TableChanges.TableChange tableChange = tableSchemas.get(tableId);
Expand All @@ -600,6 +601,17 @@ private MySqlSnapshotSplit createSnapshotSplit(
// only support one split key
String splitKey = splitKeys.get(0);
Column splitColumn = tableChange.getTable().columnWithName(splitKey);
Preconditions.checkNotNull(
splitColumn,
"Split key column "
+ splitKey
+ " not found in table "
+ tableId
+ " for job "
+ jobConfig.getJobId());
Class<?> keyClass = resolveSplitKeyClass(tableId, splitColumn, jobConfig);
Object[] splitStart = convertBounds(snapshotSplit.getSplitStart(), keyClass, objectMapper);
Object[] splitEnd = convertBounds(snapshotSplit.getSplitEnd(), keyClass, objectMapper);
RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn, false);
MySqlSnapshotSplit split =
new MySqlSnapshotSplit(
Expand Down Expand Up @@ -631,6 +643,7 @@ private Tuple2<MySqlSplit, Boolean> createBinlogSplit(
.sorted(Comparator.comparing(AbstractSourceSplit::getSplitId))
.toList();

Map<TableId, TableChanges.TableChange> tableSchemas = getTableSchemas(config);
for (SnapshotSplit split : assignedSplitLists) {
// find the min binlog offset
Map<String, String> offsetMap = split.getHighWatermark();
Expand All @@ -641,13 +654,26 @@ private Tuple2<MySqlSplit, Boolean> createBinlogSplit(
if (maxOffsetFinishSplits == null || binlogOffset.isAfter(maxOffsetFinishSplits)) {
maxOffsetFinishSplits = binlogOffset;
}
TableId tid = TableId.parse(split.getTableId());
TableChanges.TableChange tableChange = tableSchemas.get(tid);
Preconditions.checkNotNull(
tableChange, "Can not find table " + tid + " in job " + config.getJobId());
String splitKey = split.getSplitKey().get(0);
Column splitColumn = tableChange.getTable().columnWithName(splitKey);
Preconditions.checkNotNull(
splitColumn,
"Split key column "
+ splitKey
+ " not found in table "
+ tid
+ " for job "
+ config.getJobId());
Class<?> keyClass = resolveSplitKeyClass(tid, splitColumn, config);
Object[] start = convertBounds(split.getSplitStart(), keyClass, objectMapper);
Object[] end = convertBounds(split.getSplitEnd(), keyClass, objectMapper);
finishedSnapshotSplitInfos.add(
new FinishedSnapshotSplitInfo(
TableId.parse(split.getTableId()),
split.getSplitId(),
split.getSplitStart(),
split.getSplitEnd(),
binlogOffset));
tid, split.getSplitId(), start, end, binlogOffset));
}
}

Expand Down Expand Up @@ -1050,6 +1076,24 @@ private Map<TableId, TableChanges.TableChange> getTableSchemas(JobBaseConfig con
return schemas;
}

@Override
protected Class<?> probeSplitKeyClass(
TableId tableId, Column splitColumn, JobBaseConfig jobConfig) {
MySqlSourceConfig sourceConfig = getSourceConfig(jobConfig);
String sql =
String.format(
"SELECT %s FROM %s WHERE 1=0",
StatementUtils.quote(splitColumn.name()), StatementUtils.quote(tableId));
try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig);
Statement st = jdbc.connection().createStatement();
ResultSet rs = st.executeQuery(sql)) {
return Class.forName(rs.getMetaData().getColumnClassName(1));
} catch (Exception e) {
throw new RuntimeException(
"Probe split key class failed for " + tableId + "." + splitColumn.name(), e);
}
}

private Map<TableId, TableChanges.TableChange> discoverTableSchemas(JobBaseConfig config) {
MySqlSourceConfig sourceConfig = getSourceConfig(config);
try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,10 +49,13 @@
import org.apache.flink.cdc.connectors.postgres.source.offset.PostgresOffset;
import org.apache.flink.cdc.connectors.postgres.source.offset.PostgresOffsetFactory;
import org.apache.flink.cdc.connectors.postgres.source.utils.CustomPostgresSchema;
import org.apache.flink.cdc.connectors.postgres.source.utils.PostgresQueryUtils;
import org.apache.flink.cdc.connectors.postgres.source.utils.PostgresTypeUtils;
import org.apache.flink.cdc.connectors.postgres.source.utils.TableDiscoveryUtils;
import org.apache.flink.table.types.DataType;

import java.sql.ResultSet;
import java.sql.Statement;
import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
Expand Down Expand Up @@ -366,6 +369,26 @@ protected DataType fromDbzColumn(Column splitColumn) {
return PostgresTypeUtils.fromDbzColumn(splitColumn);
}

@Override
protected Class<?> probeSplitKeyClass(
TableId tableId, Column splitColumn, JobBaseConfig jobConfig) {
PostgresSourceConfig sourceConfig = getSourceConfig(jobConfig);
String sql =
String.format(
"SELECT %s FROM %s WHERE 1=0",
PostgresQueryUtils.quote(splitColumn.name()),
PostgresQueryUtils.quote(tableId));
try (JdbcConnection jdbc =
new PostgresDialect(sourceConfig).openJdbcConnection(sourceConfig);
Statement st = jdbc.connection().createStatement();
ResultSet rs = st.executeQuery(sql)) {
return Class.forName(rs.getMetaData().getColumnClassName(1));
} catch (Exception e) {
throw new RuntimeException(
"Probe split key class failed for " + tableId + "." + splitColumn.name(), e);
}
}

/**
* Why not call dialect.displayCurrentOffset(sourceConfig) ? The underlying system calls
* `txid_current()` to advance the WAL log. Here, it's just a query; retrieving the LSN is
Expand Down
Loading
Loading