diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/DebeziumUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/DebeziumUtils.java index 18e902db5aa..69819c74001 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/DebeziumUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/DebeziumUtils.java @@ -247,6 +247,9 @@ public static BinlogOffset findBinlogOffset( if (binlogFiles.isEmpty()) { return BinlogOffset.ofBinlogFilePosition("", 0); } + if (binlogFiles.size() == 1) { + return BinlogOffset.ofBinlogFilePosition(binlogFiles.get(0), 0); + } BinaryLogClient client = createBinaryClient(mySqlSourceConfig.getDbzConfiguration()); if (mySqlSourceConfig.getServerIdRange() != null) {