lvyanquan commented on code in PR #3668:
URL: https://github.com/apache/flink-cdc/pull/3668#discussion_r1905394070


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlOnLineSchemaMigrationITCase.java:
##########
@@ -0,0 +1,591 @@
+/*
+ * 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.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.PhysicalColumn;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.source.FlinkSourceProvider;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+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.BinaryRecordDataGenerator;
+import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.lifecycle.Startables;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCHEMA_CHANGE_ENABLED;
+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.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.fetchResults;
+import static 
org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.getServerId;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * IT case for Evolving MySQL schema with gh-ost/pt-osc utility. See <a
+ * href="https://github.com/github/gh-ost";>github/gh-ost</a>/<a
+ * 
href="https://docs.percona.com/percona-toolkit/pt-online-schema-change.html";>doc/pt-osc</a>
 for
+ * more details.
+ */
+public class MySqlOnLineSchemaMigrationITCase extends MySqlSourceTestBase {
+    private static final MySqlContainer MYSQL8_CONTAINER =
+            createMySqlContainer(MySqlVersion.V8_0, 
"docker/server-gtids/expire-seconds/my.cnf");
+
+    private static final String PERCONA_TOOLKIT = 
"perconalab/percona-toolkit:3.5.7";
+
+    protected static final GenericContainer<?> PERCONA_TOOLKIT_CONTAINER =
+            createPerconaToolkitContainer();
+
+    private final UniqueDatabase customerDatabase =
+            new UniqueDatabase(MYSQL8_CONTAINER, "customer", TEST_USER, 
TEST_PASSWORD);
+
+    private final StreamExecutionEnvironment env =
+            StreamExecutionEnvironment.getExecutionEnvironment();
+
+    private static final String GH_OST_DOWNLOAD_LINK =
+            
DockerClientFactory.instance().client().versionCmd().exec().getArch().equals("amd64")
+                    ? 
"https://github.com/github/gh-ost/releases/download/v1.1.6/gh-ost-binary-linux-amd64-20231207144046.tar.gz";
+                    : 
"https://github.com/github/gh-ost/releases/download/v1.1.6/gh-ost-binary-linux-arm64-20231207144046.tar.gz";;
+
+    @BeforeClass
+    public static void beforeClass() {
+        LOG.info("Starting MySql8 containers...");
+        Startables.deepStart(Stream.of(MYSQL8_CONTAINER)).join();
+        Startables.deepStart(Stream.of(PERCONA_TOOLKIT_CONTAINER)).join();
+        LOG.info("Container MySql8 is started.");
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        LOG.info("Stopping MySql8 containers...");
+        MYSQL8_CONTAINER.stop();
+        PERCONA_TOOLKIT_CONTAINER.stop();
+        LOG.info("Container MySql8 is stopped.");
+    }
+
+    @Before
+    public void before() {
+        customerDatabase.createAndInitialize();
+        TestValuesTableFactory.clearAllData();
+        env.setParallelism(4);
+        env.enableCheckpointing(200);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+    }
+
+    @After
+    public void after() {
+        customerDatabase.dropDatabase();
+    }
+
+    private static void installGhOstCli(Container<?> container) {
+        try {
+            execInContainer(
+                    container,
+                    "download gh-ost tarball",
+                    "curl",
+                    "-L",
+                    "-o",
+                    "/tmp/gh-ost.tar.gz",
+                    GH_OST_DOWNLOAD_LINK);
+            execInContainer(
+                    container, "unzip binary", "tar", "-xzvf", 
"/tmp/gh-ost.tar.gz", "-C", "/bin");
+        } catch (IOException | InterruptedException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static GenericContainer<?> createPerconaToolkitContainer() {
+        GenericContainer<?> perconaToolkit =
+                new GenericContainer<>(PERCONA_TOOLKIT)
+                        // keep container alive
+                        .withCommand("tail", "-f", "/dev/null")
+                        .withNetwork(NETWORK)
+                        .withLogConsumer(new Slf4jLogConsumer(LOG));
+        return perconaToolkit;
+    }
+
+    @Test
+    public void testGhOstSchemaMigrationFromScratch() throws Exception {
+        LOG.info("Step 1: Install gh-ost command line utility");
+        installGhOstCli(MYSQL8_CONTAINER);
+
+        LOG.info("Step 2: Start pipeline job");
+        env.setParallelism(1);
+        TableId tableId = TableId.tableId(customerDatabase.getDatabaseName(), 
"customers");
+        MySqlSourceConfigFactory configFactory =
+                new MySqlSourceConfigFactory()
+                        .hostname(MYSQL8_CONTAINER.getHost())
+                        .port(MYSQL8_CONTAINER.getDatabasePort())
+                        .username(TEST_USER)
+                        .password(TEST_PASSWORD)
+                        .databaseList(customerDatabase.getDatabaseName())
+                        .tableList(customerDatabase.getDatabaseName() + 
"\\.customers")

Review Comment:
   I think it would be more reliable to validate capturing the changes of 
entire database.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to