John Watson created FLINK-38522:
-----------------------------------

             Summary: flink-connector-mysql-cdc fails when using 
StartupOptions.timestamp() for Database connections which require SSL.
                 Key: FLINK-38522
                 URL: https://issues.apache.org/jira/browse/FLINK-38522
             Project: Flink
          Issue Type: Bug
          Components: Flink CDC
    Affects Versions: cdc-3.5.0, cdc-3.4.0
            Reporter: John Watson


When using StartupOptions.timestamp(), the MySQL CDC connector calls 
DebeziumUtils.findBinlogOffset() to locate the appropriate binlog position. 
This method creates a short-lived BinaryLogClient to scan binlog files, but 
does not configure SSL mode on the client.

If the MySQL server requires SSL connections (e.g., require_secure_transport=ON 
or SSL mode configured in connection parameters), the connection fails because 
the BinaryLogClient attempts an unencrypted connection.

 

Steps to reproduce:

  1. Configure MySQL with require_secure_transport=ON or SSL mode requirements
  2. Create a MySQL CDC source with StartupOptions.timestamp(timestampMs)
  3. Start the Flink job
  4. Connection fails in findBinlogOffset() when scanning binlog files

 

Stack trace:
{noformat}
java.lang.RuntimeException: One or more fetchers have encountered exception    
at 
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:333)
    at 
org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:228)
    at 
org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:190)
    at 
org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:444)
    at 
org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
    at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:638)
    at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:973)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:917) 
   at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:972)
    at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:951)    at 
org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:765)    at 
org.apache.flink.runtime.taskmanager.Task.run(Task.java:577)    at 
java.base/java.lang.Thread.run(Thread.java:829)Caused by: 
java.lang.RuntimeException: SplitFetcher thread 0 received unexpected exception 
while polling the records    at 
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:168)
    at 
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:117)
    at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    ... 1 moreCaused by: org.apache.flink.util.FlinkRuntimeException: 
com.github.shyiko.mysql.binlog.network.AuthenticationException: Access denied 
for user 'user'@'10.158.1.12' (using password: YES)    at 
org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.findBinlogOffset(DebeziumUtils.java:277)
    at 
org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils.initializeEffectiveOffset(BinlogOffsetUtils.java:55)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext.loadStartingOffsetState(StatefulTaskContext.java:195)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext.configure(StatefulTaskContext.java:129)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.reader.BinlogSplitReader.submitSplit(BinlogSplitReader.java:118)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.reader.BinlogSplitReader.submitSplit(BinlogSplitReader.java:72)
    at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSplitReader.pollSplitRecords(MySqlSplitReader.java:113)
    at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSplitReader.fetch(MySqlSplitReader.java:84)
    at 
org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:58)
    at 
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:165)
    ... 6 moreCaused by: 
com.github.shyiko.mysql.binlog.network.AuthenticationException: Access denied 
for user 'user'@'10.158.1.12' (using password: YES)    at 
com.github.shyiko.mysql.binlog.network.Authenticator.readResult(Authenticator.java:85)
    at 
com.github.shyiko.mysql.binlog.network.Authenticator.authenticate(Authenticator.java:70)
    at 
com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:574)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.getBinlogTimestamp(DebeziumUtils.java:333)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.searchBinlogName(DebeziumUtils.java:289)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.findBinlogOffset(DebeziumUtils.java:274)
    ... 15 more
{noformat}
 

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to