Copilot commented on code in PR #4156:
URL: https://github.com/apache/flink-cdc/pull/4156#discussion_r2917939191
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java:
##########
@@ -96,12 +97,15 @@ public class BinlogSplitReader implements
DebeziumReader<SourceRecords, MySqlSpl
private static final long READER_CLOSE_TIMEOUT = 30L;
public BinlogSplitReader(MySqlSourceConfig sourceConfig, int subtaskId) {
- this(
- new StatefulTaskContext(
- sourceConfig,
- createBinaryClient(sourceConfig.getDbzConfiguration()),
- createMySqlConnection(sourceConfig)),
- subtaskId);
+ this(createStatefulTaskContext(sourceConfig), subtaskId);
+ }
+
+ private static StatefulTaskContext
createStatefulTaskContext(MySqlSourceConfig sourceConfig) {
+ MySqlConnection connection = createMySqlConnection(sourceConfig);
+ return new StatefulTaskContext(
+ sourceConfig,
+ createBinaryClient(sourceConfig.getDbzConfiguration(),
connection),
+ connection);
Review Comment:
`createStatefulTaskContext` allocates a `MySqlConnection` and then calls
`createBinaryClient(...)`. If BinaryLogClient creation fails (e.g., due to SSL
configuration problems), the connection created just above will not be closed.
Wrap the BinaryLogClient creation in try/catch and close the connection on
failure to avoid leaking resources during reader initialization failures.
```suggestion
try {
return new StatefulTaskContext(
sourceConfig,
createBinaryClient(sourceConfig.getDbzConfiguration(),
connection),
connection);
} catch (Exception e) {
try {
connection.close();
} catch (Exception closeException) {
LOG.warn(
"Failed to close MySqlConnection after
BinaryLogClient creation failure.",
closeException);
}
throw e;
}
```
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/SnapshotSplitReaderTest.java:
##########
@@ -115,9 +117,10 @@ void testReadSingleSnapshotSplit() throws Exception {
void testReadSingleSnapshotSplitWithDotName() throws Exception {
MySqlSourceConfig sourceConfig =
getConfig(customer3_0Database, new String[] {"customers3.0"},
4);
- BinaryLogClient binaryLogClient =
-
DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration());
MySqlConnection mySqlConnection =
DebeziumUtils.createMySqlConnection(sourceConfig);
+ BinaryLogClient binaryLogClient =
+ DebeziumUtils.createBinaryClient(
+ sourceConfig.getDbzConfiguration(), mySqlConnection);
Review Comment:
`testReadSingleSnapshotSplitWithDotName` creates a new `MySqlConnection` and
`BinaryLogClient` but never closes/disconnects them. This can leak connections
across the test suite (and is more noticeable now that the BinaryLogClient
construction may interact with the connection for SSL settings). Wrap these in
try/finally (or try-with-resources for `MySqlConnection`) and ensure the
BinaryLogClient is disconnected.
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java:
##########
@@ -99,13 +100,15 @@ public class SnapshotSplitReader implements
DebeziumReader<SourceRecords, MySqlS
public SnapshotSplitReader(
MySqlSourceConfig sourceConfig, int subtaskId, SnapshotPhaseHooks
hooks) {
- this(
- new StatefulTaskContext(
- sourceConfig,
- createBinaryClient(sourceConfig.getDbzConfiguration()),
- createMySqlConnection(sourceConfig)),
- subtaskId,
- hooks);
+ this(createStatefulTaskContext(sourceConfig), subtaskId, hooks);
+ }
+
+ private static StatefulTaskContext
createStatefulTaskContext(MySqlSourceConfig sourceConfig) {
+ MySqlConnection connection = createMySqlConnection(sourceConfig);
+ return new StatefulTaskContext(
+ sourceConfig,
+ createBinaryClient(sourceConfig.getDbzConfiguration(),
connection),
+ connection);
Review Comment:
`createStatefulTaskContext` creates a `MySqlConnection` and then constructs
the `BinaryLogClient`. If `createBinaryClient(...)` throws (e.g., SSL
keystore/truststore load failure), the newly created connection will be leaked.
Consider guarding this with try/catch to close the connection on failure before
rethrowing.
```suggestion
try {
return new StatefulTaskContext(
sourceConfig,
createBinaryClient(sourceConfig.getDbzConfiguration(),
connection),
connection);
} catch (Throwable t) {
try {
connection.close();
} catch (Throwable closeEx) {
t.addSuppressed(closeEx);
}
throw t;
}
```
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/DebeziumUtils.java:
##########
@@ -242,12 +274,101 @@ private static Map<String, String> querySystemVariables(
return variables;
}
+ static SSLMode sslModeFor(MySqlConnectorConfig.SecureConnectionMode mode) {
+ try {
+ return mode == null ? null : SSLMode.valueOf(mode.name());
+ } catch (IllegalArgumentException e) {
+ throw new FlinkRuntimeException(
+ String.format("Invalid SecureConnectionMode provided: %s
", mode.name()), e);
Review Comment:
`sslModeFor(...)` throws a FlinkRuntimeException with a formatted message
that has a trailing space (`"%s "`). This makes the message slightly
inconsistent and can complicate exact-message assertions/log parsing. Consider
removing the trailing space and/or including the logical connector name for
better diagnostics.
```suggestion
String.format("Invalid SecureConnectionMode provided:
%s", mode.name()), e);
```
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceSSLConnectionITCase.java:
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.mysql.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils;
+import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig;
+import
org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory;
+import org.apache.flink.cdc.connectors.mysql.table.StartupOptions;
+import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase;
+import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+
+import com.github.shyiko.mysql.binlog.BinaryLogClient;
+import com.github.shyiko.mysql.binlog.network.SSLMode;
+import io.debezium.connector.mysql.MySqlConnection;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/** IT Tests for {@link MySqlSource}. */
+@Timeout(value = 20, unit = TimeUnit.SECONDS)
Review Comment:
The class-level `@Timeout` is 20s, but the test later uses a 30s timeout for
`hasNextFuture.get(...)`. JUnit may abort the test before the custom timeout
triggers, making the failure mode flaky/non-deterministic. Align the internal
timeout and the JUnit timeout (e.g., reduce the internal value or increase
`@Timeout`).
```suggestion
@Timeout(value = 40, unit = TimeUnit.SECONDS)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]