yuxiqian commented on code in PR #4149:
URL: https://github.com/apache/flink-cdc/pull/4149#discussion_r2434400292
##########
flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/migration/YamlJobMigrationITCase.java:
##########
@@ -99,7 +99,7 @@ void testBasicJobSubmitting() throws Exception {
}
@ParameterizedTest(name = "{0} -> SNAPSHOT")
- @EnumSource(names = {"V3_2_1", "V3_3_0", "SNAPSHOT"})
+ @EnumSource(names = {"V3_2_1", "V3_3_0", "V3_4_0", "SNAPSHOT"})
Review Comment:
We may add 3.5.0 into the matrix now
##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSink.java:
##########
@@ -33,30 +37,35 @@
import org.apache.paimon.options.Options;
import org.apache.paimon.table.sink.CommitMessageSerializer;
+import java.util.Collection;
+import java.util.UUID;
+
/**
* A {@link Sink} for Paimon. Maintain this package until Paimon has it own
sinkV2 implementation.
*/
-public class PaimonSink<InputT> implements WithPreCommitTopology<InputT,
MultiTableCommittable> {
+public class PaimonSink<InputT>
+ implements WithPreCommitTopology<InputT, MultiTableCommittable>,
+ SupportsWriterState<InputT, PaimonWriterState> {
// provided a default commit user.
public static final String DEFAULT_COMMIT_USER = "admin";
protected final Options catalogOptions;
+ /** The commitUser should be restored in state and restore it in writer. */
protected final String commitUser;
private final PaimonRecordSerializer<InputT> serializer;
public PaimonSink(Options catalogOptions, PaimonRecordSerializer<InputT>
serializer) {
- this.catalogOptions = catalogOptions;
- this.serializer = serializer;
- commitUser = DEFAULT_COMMIT_USER;
+ this(catalogOptions, DEFAULT_COMMIT_USER, serializer);
}
public PaimonSink(
Options catalogOptions, String commitUser,
PaimonRecordSerializer<InputT> serializer) {
this.catalogOptions = catalogOptions;
- this.commitUser = commitUser;
+ // generate a random commit user to avoid conflict.
+ this.commitUser = commitUser + UUID.randomUUID();
Review Comment:
Can we introduce `commit.user-prefix` as an alias of `commit.user`? Paimon
also uses this option name, and I think it describes how commitUser is created
better.
##########
flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/migration/MySqlToPaimonMigrationITCase.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.pipeline.tests.migration;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.cdc.common.test.utils.TestUtils;
+import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer;
+import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase;
+import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment;
+import org.apache.flink.cdc.pipeline.tests.utils.TarballFetcher;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.images.builder.Transferable;
+import org.testcontainers.utility.MountableFile;
+
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * E2e cases for stopping & restarting jobs of `MySQL source to Paimon sink`
from previous state.
+ */
+class MySqlToPaimonMigrationITCase extends PipelineTestEnvironment {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(MySqlToPaimonMigrationITCase.class);
+
+ private static final Duration PAIMON_TESTCASE_TIMEOUT =
Duration.ofMinutes(3);
+
+ protected UniqueDatabase mysqlInventoryDatabase;
+ private final Function<String, String> dbNameFormatter =
+ (s) -> String.format(s, mysqlInventoryDatabase.getDatabaseName());
+
+ @BeforeEach
+ public void before() throws Exception {
+ super.before();
+ mysqlInventoryDatabase =
+ new UniqueDatabase(MYSQL, "mysql_inventory", MYSQL_TEST_USER,
MYSQL_TEST_PASSWORD);
+ mysqlInventoryDatabase.createAndInitialize();
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(
+
TestUtils.getResource(getPaimonSQLConnectorResourceName())),
+ sharedVolume.toString() + "/" +
getPaimonSQLConnectorResourceName());
+ jobManager.copyFileToContainer(
+
MountableFile.forHostPath(TestUtils.getResource("flink-shade-hadoop.jar")),
+ sharedVolume.toString() + "/flink-shade-hadoop.jar");
+ }
+
+ @AfterEach
+ public void after() {
+ super.after();
+ if (mysqlInventoryDatabase != null) {
+ mysqlInventoryDatabase.dropDatabase();
+ }
+ }
+
+ @Test
+ void testBasicJobSubmitting() throws Exception {
+ String warehouse = sharedVolume.toString() + "/" + "paimon_" +
UUID.randomUUID();
+ String content =
+ String.format(
+ "source:\n"
+ + " type: mysql\n"
+ + " hostname: %s\n"
+ + " port: %d\n"
+ + " username: %s\n"
+ + " password: %s\n"
+ + " tables: %s.products\n"
+ + " server-id: 5400-5404\n"
+ + " server-time-zone: UTC\n"
+ + "\n"
+ + "sink:\n"
+ + " type: paimon\n"
+ + " catalog.properties.warehouse: %s\n"
+ + " catalog.properties.metastore:
filesystem\n"
+ + " catalog.properties.cache-enabled: false\n"
+ + "\n"
+ + "pipeline:\n"
+ + " parallelism: %d\n",
+ INTER_CONTAINER_MYSQL_ALIAS,
+ MySqlContainer.MYSQL_PORT,
+ MYSQL_TEST_USER,
+ MYSQL_TEST_PASSWORD,
+ mysqlInventoryDatabase.getDatabaseName(),
+ warehouse,
+ 4);
+ Path paimonCdcConnector =
TestUtils.getResource("paimon-cdc-pipeline-connector.jar");
+ Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar");
+ JobID jobID = submitPipelineJob(content, paimonCdcConnector,
hadoopJar);
+ Assertions.assertThat(jobID).isNotNull();
+ LOG.info("Submitted Job ID is {} ", jobID);
+ validateSinkResult(
+ warehouse,
+ mysqlInventoryDatabase.getDatabaseName(),
+ "products",
+ Arrays.asList(
+ "101, scooter, Small 2-wheel scooter, 3.14, red,
{\"key1\": \"value1\"}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}",
+ "102, car battery, 12V car battery, 8.1, white,
{\"key2\": \"value2\"}, {\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0}",
+ "103, 12-pack drill bits, 12-pack of drill bits with
sizes ranging from #40 to #3, 0.8, red, {\"key3\": \"value3\"},
{\"coordinates\":[3,3],\"type\":\"Point\",\"srid\":0}",
+ "104, hammer, 12oz carpenter's hammer, 0.75, white,
{\"key4\": \"value4\"}, {\"coordinates\":[4,4],\"type\":\"Point\",\"srid\":0}",
+ "105, hammer, 14oz carpenter's hammer, 0.875, red,
{\"k1\": \"v1\", \"k2\": \"v2\"},
{\"coordinates\":[5,5],\"type\":\"Point\",\"srid\":0}",
+ "106, hammer, 16oz carpenter's hammer, 1.0, null,
null, null",
+ "107, rocks, box of assorted rocks, 5.3, null, null,
null",
+ "108, jacket, water resistent black wind breaker, 0.1,
null, null, null",
+ "109, spare tire, 24 inch spare tire, 22.2, null,
null, null"));
+ LOG.info("Snapshot phase successfully finished.");
+
+ waitUntilJobFinished(Duration.ofSeconds(30));
+ LOG.info("Job gracefully stopped.");
+ }
+
+ @ParameterizedTest(name = "{0} -> SNAPSHOT")
+ @EnumSource(names = {"V3_2_1", "V3_3_0", "V3_4_0", "SNAPSHOT"})
Review Comment:
Ditto
--
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]