Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -27,6 +27,7 @@
import com.github.shyiko.mysql.binlog.event.EventData;
import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
import com.github.shyiko.mysql.binlog.event.RotateEventData;
import com.github.shyiko.mysql.binlog.network.SSLMode;
import io.debezium.config.Configuration;
import io.debezium.connector.mysql.MySqlConnection;
import io.debezium.connector.mysql.MySqlConnectorConfig;
Expand Down Expand Up @@ -242,12 +243,30 @@ private static Map<String, String> querySystemVariables(
return variables;
}

static SSLMode sslModeFor(MySqlConnectorConfig.SecureConnectionMode mode) {
switch (mode) {
case DISABLED:
return SSLMode.DISABLED;
case PREFERRED:
return SSLMode.PREFERRED;
case REQUIRED:
return SSLMode.REQUIRED;
case VERIFY_CA:
return SSLMode.VERIFY_CA;
case VERIFY_IDENTITY:
return SSLMode.VERIFY_IDENTITY;
}
return null;
}

public static BinlogOffset findBinlogOffset(
long targetMs, MySqlConnection connection, MySqlSourceConfig mySqlSourceConfig) {
MySqlConnection.MySqlConnectionConfiguration config = connection.connectionConfig();
BinaryLogClient client =
Copy link
Contributor

@lvyanquan lvyanquan Dec 5, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can centralize the SSL configuration in the DebeziumUtils#createBinaryClient method,


and based on the implementation of MySqlStreamingChangeEventSource, we also need to add the following code to set SslSocketFactory:
client.setSSLMode(sslModeFor(connectorConfig.sslMode()));
if (connectorConfig.sslModeEnabled()) {
SSLSocketFactory sslSocketFactory =
getBinlogSslSocketFactory(connectorConfig, connection);
if (sslSocketFactory != null) {
client.setSslSocketFactory(sslSocketFactory);
}
}

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review and recommendation @lvyanquan !
I have now updated this PR:

  • ssl config now centralised under DebeziumUtils#createBinaryClient
  • updated BinlogSplitReader and SnapshotSplitReader to use the new version of createBinaryClient
  • added sslSocketFactory config, similar to what is used in flink-cdc/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java
  • updated iTest to include testing sslSocketFactory.

new BinaryLogClient(
config.hostname(), config.port(), config.username(), config.password());
client.setSSLMode(sslModeFor(config.sslMode()));

if (mySqlSourceConfig.getServerIdRange() != null) {
client.setServerId(mySqlSourceConfig.getServerIdRange().getStartServerId());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,19 @@
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory;
import org.apache.flink.cdc.connectors.mysql.table.StartupOptions;

import com.github.shyiko.mysql.binlog.network.SSLMode;
import io.debezium.connector.mysql.MySqlConnection;
import io.debezium.connector.mysql.MySqlConnectorConfig;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;

import java.time.Duration;
import java.time.ZoneId;
import java.util.Properties;
import java.util.stream.Stream;

/** Tests for {@link org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils}. */
class DebeziumUtilsTest {
Expand Down Expand Up @@ -83,6 +89,26 @@ private MySqlSourceConfig getConfig(Properties jdbcProperties) {
.createConfig(0);
}

@ParameterizedTest
@MethodSource("sslModeProvider")
void testSslModeConversion(MySqlConnectorConfig.SecureConnectionMode input, SSLMode expected) {
SSLMode actual = DebeziumUtils.sslModeFor(input);
Assertions.assertThat(actual).isEqualTo(expected);
}

static Stream<Arguments> sslModeProvider() {
return Stream.of(
Arguments.of(MySqlConnectorConfig.SecureConnectionMode.DISABLED, SSLMode.DISABLED),
Arguments.of(
MySqlConnectorConfig.SecureConnectionMode.PREFERRED, SSLMode.PREFERRED),
Arguments.of(MySqlConnectorConfig.SecureConnectionMode.REQUIRED, SSLMode.REQUIRED),
Arguments.of(
MySqlConnectorConfig.SecureConnectionMode.VERIFY_CA, SSLMode.VERIFY_CA),
Arguments.of(
MySqlConnectorConfig.SecureConnectionMode.VERIFY_IDENTITY,
SSLMode.VERIFY_IDENTITY));
}

private void assertJdbcUrl(String expected, String actual) {
// Compare after splitting to avoid the orderless jdbc parameters in jdbc url at Java 11
String[] expectedParam = expected.split("&");
Expand Down