Skip to content
Closed
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 @@ -67,22 +67,26 @@ public EventSourceProvider getEventSourceProvider() {
RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS.name(),
false);

boolean isTableIdCaseInsensitive = MySqlSchemaUtils.isTableIdCaseInsensitive(sourceConfig);
MySqlEventDeserializer deserializer =
new MySqlEventDeserializer(
DebeziumChangelogMode.ALL,
sourceConfig.isIncludeSchemaChanges(),
readableMetadataList,
includeComments,
sourceConfig.isTreatTinyInt1AsBoolean(),
MySqlSchemaUtils.isTableIdCaseInsensitive(sourceConfig));
isTableIdCaseInsensitive);

MySqlSource<Event> source =
new MySqlSource<>(
configFactory,
deserializer,
(sourceReaderMetrics, sourceConfig) ->
new MySqlPipelineRecordEmitter(
deserializer, sourceReaderMetrics, sourceConfig));
deserializer,
sourceReaderMetrics,
sourceConfig,
isTableIdCaseInsensitive));

return FlinkSourceProvider.of(source);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,9 +55,11 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;

import static org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.openJdbcConnection;
import static org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils.getTableId;
Expand All @@ -80,6 +82,7 @@ public class MySqlPipelineRecordEmitter extends MySqlRecordEmitter<Event> {
// Used when startup mode is snapshot
private boolean shouldEmitAllCreateTableEventsInSnapshotMode = true;
private boolean isBounded = false;
private final boolean isTableIdCaseInsensitive;

private final DebeziumDeserializationSchema<Event> debeziumDeserializationSchema;

Expand All @@ -88,7 +91,8 @@ public class MySqlPipelineRecordEmitter extends MySqlRecordEmitter<Event> {
public MySqlPipelineRecordEmitter(
DebeziumDeserializationSchema<Event> debeziumDeserializationSchema,
MySqlSourceReaderMetrics sourceReaderMetrics,
MySqlSourceConfig sourceConfig) {
MySqlSourceConfig sourceConfig,
boolean isTableIdCaseInsensitive) {
super(
debeziumDeserializationSchema,
sourceReaderMetrics,
Expand All @@ -102,6 +106,7 @@ public MySqlPipelineRecordEmitter(
((DebeziumEventDeserializationSchema) debeziumDeserializationSchema)
.getCreateTableEventCache();
this.isBounded = StartupOptions.snapshot().equals(sourceConfig.getStartupOptions());
this.isTableIdCaseInsensitive = isTableIdCaseInsensitive;
}

@Override
Expand Down Expand Up @@ -261,7 +266,10 @@ private Schema buildSchemaFromTable(Table table) {
for (int i = 0; i < columns.size(); i++) {
Column column = columns.get(i);

String colName = column.name();
String colName =
this.isTableIdCaseInsensitive
? column.name().toLowerCase(Locale.ROOT)
: column.name();
DataType dataType =
MySqlTypeUtils.fromDbzColumn(column, sourceConfig.isTreatTinyInt1AsBoolean());
if (!column.isOptional()) {
Expand All @@ -277,6 +285,12 @@ private Schema buildSchemaFromTable(Table table) {

List<String> primaryKey = table.primaryKeyColumnNames();
if (Objects.nonNull(primaryKey) && !primaryKey.isEmpty()) {
if (this.isTableIdCaseInsensitive) {
primaryKey =
primaryKey.stream()
.map(key -> key.toLowerCase(Locale.ROOT))
.collect(Collectors.toList());
}
tableBuilder.primaryKey(primaryKey);
}
return tableBuilder.build();
Expand Down