-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[FLINK-36520] MySQL CDC Supports connecting to MySQL server 8.4+ #3666
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
277 changes: 277 additions & 0 deletions
277
...t/java/org/apache/flink/cdc/connectors/mysql/source/MySqlPipelineCompatibilityITCase.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,277 @@ | ||
| /* | ||
| * 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.common.event.DataChangeEvent; | ||
| import org.apache.flink.cdc.common.event.Event; | ||
| import org.apache.flink.cdc.common.source.FlinkSourceProvider; | ||
| import org.apache.flink.cdc.connectors.mysql.factory.MySqlDataSourceFactory; | ||
| 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.MySqlContainer; | ||
| import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; | ||
| import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; | ||
| import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.streaming.util.RestartStrategyUtils; | ||
| import org.apache.flink.util.CloseableIterator; | ||
|
|
||
| import org.junit.jupiter.api.AfterEach; | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.params.ParameterizedClass; | ||
| import org.junit.jupiter.params.provider.EnumSource; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import org.testcontainers.containers.output.Slf4jLogConsumer; | ||
| import org.testcontainers.lifecycle.Startables; | ||
|
|
||
| import java.io.File; | ||
| import java.nio.charset.StandardCharsets; | ||
| import java.nio.file.Files; | ||
| import java.nio.file.Path; | ||
| import java.nio.file.Paths; | ||
| import java.nio.file.StandardOpenOption; | ||
| import java.sql.Connection; | ||
| import java.sql.Statement; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.Objects; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_PASSWORD; | ||
| import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_USER; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
|
|
||
| /** | ||
| * Integration tests to check MySQL pipeline connector works well with different MySQL server | ||
| * versions. | ||
| */ | ||
| @ParameterizedClass | ||
| @EnumSource( | ||
| value = MySqlVersion.class, | ||
| names = {"V5_7", "V8_0", "V8_4"}) | ||
| class MySqlPipelineCompatibilityITCase { | ||
|
|
||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(MySqlPipelineCompatibilityITCase.class); | ||
|
|
||
| private static Path tempFolder; | ||
| private static File resourceFolder; | ||
|
|
||
| private final MySqlVersion version; | ||
| private final MySqlContainer mySqlContainer; | ||
| private final UniqueDatabase testDatabase; | ||
|
|
||
| private final StreamExecutionEnvironment env = | ||
| StreamExecutionEnvironment.getExecutionEnvironment(); | ||
|
|
||
| MySqlPipelineCompatibilityITCase(MySqlVersion version) { | ||
| this.version = version; | ||
| this.mySqlContainer = | ||
| (MySqlContainer) | ||
| new MySqlContainer(version) | ||
| .withConfigurationOverride(buildCustomMySqlConfig(version)) | ||
| .withSetupSQL("docker/setup.sql") | ||
| .withDatabaseName("flink-test") | ||
| .withUsername("flinkuser") | ||
| .withPassword("flinkpw") | ||
| .withLogConsumer(new Slf4jLogConsumer(LOG)); | ||
| this.testDatabase = | ||
| new UniqueDatabase(mySqlContainer, "inventory", TEST_USER, TEST_PASSWORD); | ||
| } | ||
|
|
||
| @BeforeEach | ||
| void setup() throws Exception { | ||
| // Initialize static resources if needed | ||
| if (resourceFolder == null) { | ||
| resourceFolder = | ||
| Paths.get( | ||
| Objects.requireNonNull( | ||
| MySqlPipelineCompatibilityITCase.class | ||
| .getClassLoader() | ||
| .getResource(".")) | ||
| .toURI()) | ||
| .toFile(); | ||
| tempFolder = Files.createTempDirectory(resourceFolder.toPath(), "mysql-config"); | ||
| } | ||
|
|
||
| env.setParallelism(4); | ||
| env.enableCheckpointing(200); | ||
| RestartStrategyUtils.configureNoRestartStrategy(env); | ||
|
|
||
| LOG.info("Starting container for MySQL {}...", version.getVersion()); | ||
| Startables.deepStart(Stream.of(mySqlContainer)).join(); | ||
| LOG.info("Container is started."); | ||
|
|
||
| testDatabase.createAndInitialize(); | ||
| } | ||
|
|
||
| @AfterEach | ||
| void tearDown() { | ||
| try { | ||
| testDatabase.dropDatabase(); | ||
| } catch (IllegalStateException e) { | ||
| LOG.warn("Failed to drop test database during teardown.", e); | ||
| } | ||
| if (mySqlContainer != null) { | ||
| LOG.info("Stopping container for MySQL {}...", version.getVersion()); | ||
| mySqlContainer.stop(); | ||
| LOG.info("Container is stopped."); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| void testSnapshotRead() throws Exception { | ||
| MySqlSourceConfigFactory configFactory = | ||
| new MySqlSourceConfigFactory() | ||
| .hostname(mySqlContainer.getHost()) | ||
| .port(mySqlContainer.getDatabasePort()) | ||
| .username(TEST_USER) | ||
| .password(TEST_PASSWORD) | ||
| .databaseList(testDatabase.getDatabaseName()) | ||
| .tableList(testDatabase.getDatabaseName() + ".products") | ||
| .startupOptions(StartupOptions.initial()) | ||
| .serverId(getServerId(env.getParallelism())) | ||
| .serverTimeZone("UTC"); | ||
|
|
||
| FlinkSourceProvider sourceProvider = | ||
| (FlinkSourceProvider) new MySqlDataSource(configFactory).getEventSourceProvider(); | ||
| CloseableIterator<Event> events = | ||
| env.fromSource( | ||
| sourceProvider.getSource(), | ||
| WatermarkStrategy.noWatermarks(), | ||
| MySqlDataSourceFactory.IDENTIFIER, | ||
| new EventTypeInfo()) | ||
| .executeAndCollect(); | ||
|
|
||
| List<Event> snapshotEvents = fetchEvents(events, 9); | ||
|
|
||
| assertThat(snapshotEvents).hasSize(9); | ||
| assertThat(snapshotEvents.stream().filter(e -> e instanceof DataChangeEvent)).hasSize(9); | ||
|
|
||
| events.close(); | ||
| } | ||
|
|
||
| @Test | ||
| void testBinlogRead() throws Exception { | ||
| MySqlSourceConfigFactory configFactory = | ||
| new MySqlSourceConfigFactory() | ||
| .hostname(mySqlContainer.getHost()) | ||
| .port(mySqlContainer.getDatabasePort()) | ||
| .username(TEST_USER) | ||
| .password(TEST_PASSWORD) | ||
| .databaseList(testDatabase.getDatabaseName()) | ||
| .tableList(testDatabase.getDatabaseName() + ".products") | ||
| .startupOptions(StartupOptions.initial()) | ||
| .serverId(getServerId(env.getParallelism())) | ||
| .serverTimeZone("UTC"); | ||
|
|
||
| FlinkSourceProvider sourceProvider = | ||
| (FlinkSourceProvider) new MySqlDataSource(configFactory).getEventSourceProvider(); | ||
| CloseableIterator<Event> events = | ||
| env.fromSource( | ||
| sourceProvider.getSource(), | ||
| WatermarkStrategy.noWatermarks(), | ||
| MySqlDataSourceFactory.IDENTIFIER, | ||
| new EventTypeInfo()) | ||
| .executeAndCollect(); | ||
|
|
||
| fetchEvents(events, 9); | ||
|
|
||
| try (Connection connection = testDatabase.getJdbcConnection(); | ||
| Statement statement = connection.createStatement()) { | ||
| statement.execute( | ||
| String.format( | ||
| "INSERT INTO `%s`.`products` VALUES (default,'test_product','desc',1.0);", | ||
| testDatabase.getDatabaseName())); | ||
| } | ||
|
|
||
| List<Event> binlogEvents = fetchEvents(events, 1); | ||
| assertThat(binlogEvents).hasSize(1); | ||
| assertThat(binlogEvents.get(0)).isInstanceOf(DataChangeEvent.class); | ||
|
|
||
| events.close(); | ||
| } | ||
|
|
||
| private String getServerId(int parallelism) { | ||
| int serverId = (int) (Math.random() * 100) + 5400; | ||
| return serverId + "-" + (serverId + parallelism); | ||
| } | ||
|
|
||
| private List<Event> fetchEvents(CloseableIterator<Event> iterator, int count) { | ||
| List<Event> events = new ArrayList<>(); | ||
| while (count > 0 && iterator.hasNext()) { | ||
| Event event = iterator.next(); | ||
| if (event instanceof DataChangeEvent) { | ||
| events.add(event); | ||
| count--; | ||
| } | ||
| } | ||
| return events; | ||
| } | ||
|
|
||
| private String buildCustomMySqlConfig(MySqlVersion version) { | ||
| try { | ||
| if (resourceFolder == null) { | ||
| resourceFolder = | ||
| Paths.get( | ||
| Objects.requireNonNull( | ||
| MySqlPipelineCompatibilityITCase.class | ||
| .getClassLoader() | ||
| .getResource(".")) | ||
| .toURI()) | ||
| .toFile(); | ||
| tempFolder = Files.createTempDirectory(resourceFolder.toPath(), "mysql-config"); | ||
| } | ||
| // Create version-specific directory to avoid conflicts | ||
| Path versionDir = | ||
| Files.createDirectories( | ||
| Paths.get( | ||
| tempFolder.toString(), version.getVersion().replace(".", "_"))); | ||
| Path cnf = Paths.get(versionDir.toString(), "my.cnf"); | ||
| // Check if file already exists to avoid FileAlreadyExistsException | ||
| if (!Files.exists(cnf)) { | ||
| Files.createFile(cnf); | ||
| } | ||
| StringBuilder mysqlConfBuilder = new StringBuilder(); | ||
| mysqlConfBuilder.append( | ||
| "[mysqld]\n" | ||
| + "binlog_format = row\n" | ||
| + "log_bin = mysql-bin\n" | ||
| + "server-id = 223344\n" | ||
| + "binlog_row_image = FULL\n" | ||
| + "gtid-mode = OFF\n"); | ||
|
|
||
| if (version == MySqlVersion.V8_0 || version == MySqlVersion.V8_4) { | ||
| mysqlConfBuilder.append("secure_file_priv=/var/lib/mysql\n"); | ||
| } | ||
|
|
||
| Files.write( | ||
| cnf, | ||
| Collections.singleton(mysqlConfBuilder.toString()), | ||
| StandardCharsets.UTF_8, | ||
| StandardOpenOption.TRUNCATE_EXISTING); | ||
| return Paths.get(resourceFolder.getAbsolutePath()).relativize(cnf).toString(); | ||
| } catch (Exception e) { | ||
| throw new RuntimeException("Failed to create my.cnf file.", e); | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.