Skip to content
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

[FLINK-37306][tests] Fix MySQLSourceITCase failure in non-UTC timezone #3917

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -276,6 +276,11 @@ protected UniqueDatabase getUniqueDatabase(MySqlContainer mySqlContainer) {
}

private String buildMySqlConfigWithTimezone(String timezone) {
// JVM timezone is in "GMT+XX:XX" or "GMT-XX:XX" format
// while MySQL configuration file requires "+XX:XX" or "-XX:XX"
if (timezone.startsWith("GMT")) {
timezone = timezone.substring(3);
}
try {
File folder = tempFolder.newFolder(String.valueOf(UUID.randomUUID()));
Path cnf = Files.createFile(Paths.get(folder.getPath(), "my.cnf"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.cdc.common.utils.TestCaseUtils;
import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils;
import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer;
import org.apache.flink.cdc.connectors.mysql.testutils.TestTable;
import org.apache.flink.cdc.connectors.mysql.testutils.TestTableSchemas;
import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase;
Expand All @@ -38,61 +37,34 @@
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
import org.apache.flink.streaming.api.operators.collect.CollectStreamSink;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.util.FlinkRuntimeException;

import io.debezium.connector.mysql.MySqlConnection;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
import org.junit.rules.TemporaryFolder;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.output.Slf4jLogConsumer;

import java.io.File;
import java.lang.reflect.Field;
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.time.ZoneId;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Random;
import java.util.UUID;

/**
* Integration tests for handling schema changes regard to renaming multiple tables within a single
* statement.
*/
public class MySqlMultipleTablesRenamingITCase {
public class MySqlMultipleTablesRenamingITCase extends MySqlSourceTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(MySqlMultipleTablesRenamingITCase.class);
@RegisterExtension static MiniClusterExtension miniCluster = new MiniClusterExtension();

@SuppressWarnings("unchecked")
private static final MySqlContainer MYSQL_CONTAINER =
(MySqlContainer)
new MySqlContainer()
.withConfigurationOverride(
buildMySqlConfigWithTimezone(
getResourceFolder(), getSystemTimeZone()))
.withSetupSQL("docker/setup.sql")
.withDatabaseName("flink-test")
.withUsername("flinkuser")
.withPassword("flinkpw")
.withLogConsumer(new Slf4jLogConsumer(LOG));

private final UniqueDatabase customDatabase =
new UniqueDatabase(MYSQL_CONTAINER, "customer", "mysqluser", "mysqlpw");
Expand All @@ -101,25 +73,15 @@ public class MySqlMultipleTablesRenamingITCase {

private MySqlConnection connection;

@BeforeAll
public static void before() throws Exception {
MYSQL_CONTAINER.start();
}

@AfterAll
public static void after() throws Exception {
MYSQL_CONTAINER.stop();
}

@BeforeEach
void prepare() throws Exception {
@Before
public void prepare() throws Exception {
connection = getConnection();
customDatabase.createAndInitialize();
flushLogs();
}

@AfterEach
void tearDown() throws Exception {
@After
public void tearDown() throws Exception {
customDatabase.dropDatabase();
connection.close();
}
Expand All @@ -146,7 +108,7 @@ void tearDown() throws Exception {
* during schema updates.
*/
@Test
void testRenameTablesWithinSingleStatement() throws Exception {
public void testRenameTablesWithinSingleStatement() throws Exception {
// Build Flink job
StreamExecutionEnvironment env = getExecutionEnvironment();
MySqlSource<String> source = getSourceBuilder().build();
Expand Down Expand Up @@ -269,6 +231,8 @@ private MySqlSourceBuilder<String> getSourceBuilder() {
.password(customDatabase.getPassword())
.databaseList(customDatabase.getDatabaseName())
.tableList(customers.getTableId())
.serverId(getServerId())
.serverTimeZone("UTC")
.deserializer(new JsonDebeziumDeserializationSchema());
}

Expand Down Expand Up @@ -323,50 +287,6 @@ private static List<String> fetchRow(Iterator<String> iter, int size) {
return rows;
}

private static String buildMySqlConfigWithTimezone(File resourceDirectory, String timezone) {
try {
TemporaryFolder tempFolder = new TemporaryFolder(resourceDirectory);
tempFolder.create();
File folder = tempFolder.newFolder(String.valueOf(UUID.randomUUID()));
Path cnf = Files.createFile(Paths.get(folder.getPath(), "my.cnf"));
String mysqldConf =
"[mysqld]\n"
+ "binlog_format = row\n"
+ "log_bin = mysql-bin\n"
+ "server-id = 223344\n"
+ "binlog_row_image = FULL\n"
+ "gtid_mode = on\n"
+ "enforce_gtid_consistency = on\n";
String timezoneConf = "default-time_zone = '" + timezone + "'\n";
Files.write(
cnf,
Collections.singleton(mysqldConf + timezoneConf),
StandardCharsets.UTF_8,
StandardOpenOption.APPEND);
return Paths.get(resourceDirectory.getAbsolutePath()).relativize(cnf).toString();
} catch (Exception e) {
throw new RuntimeException("Failed to create my.cnf file.", e);
}
}

private static File getResourceFolder() {
try {
return Paths.get(
Objects.requireNonNull(
SpecificStartingOffsetITCase.class
.getClassLoader()
.getResource("."))
.toURI())
.toFile();
} catch (Exception e) {
throw new FlinkRuntimeException("Get Resource File Directory fail");
}
}

private static String getSystemTimeZone() {
return ZoneId.systemDefault().toString();
}

private void setupSavepoint(StreamExecutionEnvironment env, String savepointPath)
throws Exception {
// restore from savepoint
Expand All @@ -392,4 +312,10 @@ private StreamExecutionEnvironment getExecutionEnvironment() {
env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
return env;
}

private String getServerId() {
final Random random = new Random();
int serverId = random.nextInt(100) + 5400;
return serverId + "-" + (serverId + DEFAULT_PARALLELISM);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -714,6 +714,7 @@ public void testSourceMetrics() throws Exception {
.password(customDatabase.getPassword())
.deserializer(new StringDebeziumDeserializationSchema())
.serverId(getServerId())
.serverTimeZone("UTC")
.build();
DataStreamSource<String> stream =
env.fromSource(source, WatermarkStrategy.noWatermarks(), "MySQL CDC Source");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,11 @@ private List<String> fetchRowData(
}

private static String buildMySqlConfigWithTimezone(File resourceDirectory, String timezone) {
// JVM timezone is in "GMT+XX:XX" or "GMT-XX:XX" format
// while MySQL configuration file requires "+XX:XX" or "-XX:XX"
if (timezone.startsWith("GMT")) {
timezone = timezone.substring(3);
}
try {
TemporaryFolder tempFolder = new TemporaryFolder(resourceDirectory);
tempFolder.create();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,11 @@ private static List<String> fetchRows(Iterator<Row> iter, int size) {
}

private String buildMySqlConfigWithTimezone(String timezone) {
// JVM timezone is in "GMT+XX:XX" or "GMT-XX:XX" format
// while MySQL configuration file requires "+XX:XX" or "-XX:XX"
if (timezone.startsWith("GMT")) {
timezone = timezone.substring(3);
}
try {
File folder = tempFolder.newFolder(String.valueOf(UUID.randomUUID()));
Path cnf = Files.createFile(Paths.get(folder.getPath(), "my.cnf"));
Expand Down
Loading