From a901f8b44fc4e8f46b0a568834a7a90c983d1f56 Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Thu, 18 Jan 2024 11:22:38 +0800 Subject: [PATCH] [cdc] Refactor kafka cdc test (#2672) --- .../KafkaCanalSyncTableActionITCase.java | 36 +- ...KafkaDebeziumSyncDatabaseActionITCase.java | 74 ++ .../KafkaDebeziumSyncTableActionITCase.java | 247 ++----- ...beziumWithSchemaSyncTableActionITCase.java | 228 ------ .../KafkaMaxwellSyncDatabaseActionITCase.java | 429 +---------- .../KafkaMaxwellSyncTableActionITCase.java | 479 +------------ .../KafkaOggSyncDatabaseActionITCase.java | 457 +----------- .../kafka/KafkaOggSyncTableActionITCase.java | 531 +------------- .../kafka/KafkaSyncDatabaseActionITCase.java | 556 +++++++++++++++ .../cdc/kafka/KafkaSyncTableActionITCase.java | 666 ++++++++++++++++++ .../include/topic0/debezium-data-1.txt | 22 + .../prefixsuffix/topic0/debezium-data-1.txt | 20 + .../prefixsuffix/topic0/debezium-data-2.txt | 20 + .../prefixsuffix/topic1/debezium-data-1.txt | 20 + .../prefixsuffix/topic1/debezium-data-2.txt | 20 + .../topic0/debezium-data-1.txt | 20 + .../topic0/debezium-data-2.txt | 20 + .../topic1/debezium-data-1.txt | 20 + .../topic1/debezium-data-2.txt | 20 + .../table/computedcolumn/debezium-data-1.txt | 2 +- .../table/startupmode/debezium-data-1.txt | 20 + .../table/startupmode/debezium-data-2.txt | 20 + .../database/case-insensitive/ogg-data-1.txt | 4 +- .../database/include/topic0/ogg-data-1.txt | 8 +- .../prefixsuffix/topic0/ogg-data-1.txt | 4 +- .../prefixsuffix/topic0/ogg-data-2.txt | 4 +- .../prefixsuffix/topic1/ogg-data-1.txt | 4 +- .../prefixsuffix/topic1/ogg-data-2.txt | 4 +- .../schemaevolution/topic0/ogg-data-1.txt | 4 +- .../schemaevolution/topic0/ogg-data-2.txt | 4 +- .../schemaevolution/topic1/ogg-data-1.txt | 4 +- .../schemaevolution/topic1/ogg-data-2.txt | 4 +- .../kafka/ogg/table/event/event-delete.txt | 2 +- .../kafka/ogg/table/event/event-insert.txt | 6 +- .../kafka/ogg/table/event/event-update.txt | 2 +- .../ogg/table/schemaevolution/ogg-data-1.txt | 4 +- .../ogg/table/schemaevolution/ogg-data-2.txt | 2 +- .../ogg/table/schemaevolution/ogg-data-3.txt | 4 +- .../ogg/table/startupmode/ogg-data-1.txt | 4 +- .../ogg/table/startupmode/ogg-data-2.txt | 2 +- 40 files changed, 1649 insertions(+), 2348 deletions(-) create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumWithSchemaSyncTableActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index e81ed986d0ad..bd74864e4abb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -20,8 +20,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.FileSystemCatalogOptions; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.table.AbstractFileStoreTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -57,7 +55,9 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT cases for {@link KafkaSyncTableAction}. */ -public class KafkaCanalSyncTableActionITCase extends KafkaActionITCaseBase { +public class KafkaCanalSyncTableActionITCase extends KafkaSyncTableActionITCase { + + private static final String CANAL = "canal"; @Test @Timeout(60) @@ -1247,34 +1247,6 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - String topic = "watermark"; - createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, readLines("kafka/canal/table/watermark/canal-data-1.txt")); - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "canal-json"); - kafkaConfig.put(TOPIC.key(), topic); - - Map config = getBasicTableConfig(); - config.put("tag.automatic-creation", "watermark"); - config.put("tag.creation-period", "hourly"); - config.put("scan.watermark.alignment.group", "alignment-group-1"); - config.put("scan.watermark.alignment.max-drift", "20 s"); - config.put("scan.watermark.alignment.update-interval", "1 s"); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); - runActionWithDefaultEnv(action); - - AbstractFileStoreTable table = - (AbstractFileStoreTable) catalog.getTable(new Identifier(database, tableName)); - while (true) { - if (table.snapshotManager().snapshotCount() > 0 - && table.snapshotManager().latestSnapshot().watermark() - != -9223372036854775808L) { - return; - } - Thread.sleep(1000); - } + testWaterMarkSyncTable(CANAL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java new file mode 100644 index 000000000000..94efa60c71e4 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseActionITCase.java @@ -0,0 +1,74 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +/** IT cases for {@link KafkaSyncDatabaseAction}. */ +public class KafkaDebeziumSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { + private static final String DEBEZIUM = "debezium"; + + @Test + @Timeout(60) + public void testSchemaEvolutionMultiTopic() throws Exception { + testSchemaEvolutionMultiTopic(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testSchemaEvolutionOneTopic() throws Exception { + testSchemaEvolutionOneTopic(DEBEZIUM); + } + + @Test + public void testTopicIsEmpty() { + testTopicIsEmpty(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testTableAffixMultiTopic() throws Exception { + testTableAffixMultiTopic(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testTableAffixOneTopic() throws Exception { + testTableAffixOneTopic(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testIncludingTables() throws Exception { + testIncludingTables(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testExcludingTables() throws Exception { + testExcludingTables(DEBEZIUM); + } + + @Test + @Timeout(60) + public void testIncludingAndExcludingTables() throws Exception { + testIncludingAndExcludingTables(DEBEZIUM); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index d4d69e6e8ac7..bb90617613c7 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -18,240 +18,77 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.table.AbstractFileStoreTable; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaDebeziumSyncTableActionITCase extends KafkaSyncTableActionITCase { -/** IT cases for {@link KafkaDebeziumSyncTableActionITCase}. */ -public class KafkaDebeziumSyncTableActionITCase extends KafkaActionITCaseBase { + private static final String DEBEZIUM = "debezium"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution"); + runSingleTableSchemaEvolution("schemaevolution", DEBEZIUM); } - private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { - final String topic = "schema_evolution"; - createTestTopic(topic, 1, 1); - // ---------- Write the debezium json into Kafka ------------------- - List lines = - readLines(String.format("kafka/debezium/table/%s/debezium-data-1.txt", sourceDir)); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topic, sourceDir); + @Test + @Timeout(60) + public void testNotSupportFormat() throws Exception { + testNotSupportFormat(DEBEZIUM); } - private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exception { - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format( - "kafka/debezium/table/%s/debezium-data-2.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", - "+I[102, car battery, 12V car battery, 8.1, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format( - "kafka/debezium/table/%s/debezium-data-3.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age", "address"}); - expected = - Arrays.asList( - "+I[102, car battery, 12V car battery, 8.1, NULL, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18, NULL]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24, NULL]", - "+I[105, hammer, 14oz carpenter's hammer, 0.875, NULL, Beijing]", - "+I[107, rocks, box of assorted rocks, 5.3, NULL, NULL]"); - waitForResult(expected, table, rowType, primaryKeys); + @Test + @Timeout(60) + public void testAssertSchemaCompatible() throws Exception { + testAssertSchemaCompatible(DEBEZIUM); } @Test @Timeout(60) - public void testComputedColumn() throws Exception { - String topic = "computed_column"; - createTestTopic(topic, 1, 1); - - List lines = readLines("kafka/debezium/table/computedcolumn/debezium-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPartitionKeys("_year") - .withPrimaryKeys("id", "_year") - .withComputedColumnArgs("_year=year(date)") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.INT().notNull() - }, - new String[] {"id", "date", "_year"}); - waitForResult( - Collections.singletonList("+I[101, 2023-03-23, 2023]"), - getFileStoreTable(tableName), - rowType, - Arrays.asList("id", "_year")); + public void testStarUpOptionSpecific() throws Exception { + testStarUpOptionSpecific(DEBEZIUM); } @Test @Timeout(60) - public void testRecordWithNestedDataType() throws Exception { - String topic = "nested_type"; - createTestTopic(topic, 1, 1); + public void testStarUpOptionLatest() throws Exception { + testStarUpOptionLatest(DEBEZIUM); + } - List lines = readLines("kafka/debezium/table/nestedtype/debezium-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } + @Test + @Timeout(60) + public void testStarUpOptionTimestamp() throws Exception { + testStarUpOptionTimestamp(DEBEZIUM); + } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); + @Test + @Timeout(60) + public void testStarUpOptionEarliest() throws Exception { + testStarUpOptionEarliest(DEBEZIUM); + } - FileStoreTable table = getFileStoreTable(tableName); + @Test + @Timeout(60) + public void testStarUpOptionGroup() throws Exception { + testStarUpOptionGroup(DEBEZIUM); + } - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), DataTypes.STRING(), DataTypes.STRING() - }, - new String[] {"id", "name", "row"}); - List primaryKeys = Collections.singletonList("id"); - List expected = - Collections.singletonList("+I[101, hammer, {\"row_key\":\"value\"}]"); - waitForResult(expected, table, rowType, primaryKeys); + @Test + @Timeout(60) + public void testComputedColumn() throws Exception { + testComputedColumn(DEBEZIUM); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - String topic = "watermark"; - createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, readLines("kafka/debezium/table/watermark/debezium-data-1.txt")); - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put(TOPIC.key(), topic); - - Map config = getBasicTableConfig(); - config.put("tag.automatic-creation", "watermark"); - config.put("tag.creation-period", "hourly"); - config.put("scan.watermark.alignment.group", "alignment-group-1"); - config.put("scan.watermark.alignment.max-drift", "20 s"); - config.put("scan.watermark.alignment.update-interval", "1 s"); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); - runActionWithDefaultEnv(action); + testWaterMarkSyncTable(DEBEZIUM); + } - AbstractFileStoreTable table = - (AbstractFileStoreTable) catalog.getTable(new Identifier(database, tableName)); - while (true) { - if (table.snapshotManager().snapshotCount() > 0 - && table.snapshotManager().latestSnapshot().watermark() - != -9223372036854775808L) { - return; - } - Thread.sleep(1000); - } + @Test + @Timeout(60) + public void testKafkaBuildSchemaWithDelete() throws Exception { + testKafkaBuildSchemaWithDelete(DEBEZIUM); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumWithSchemaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumWithSchemaSyncTableActionITCase.java deleted file mode 100644 index ff156be5c286..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumWithSchemaSyncTableActionITCase.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * 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.paimon.flink.action.cdc.kafka; - -import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; -import org.apache.paimon.flink.action.cdc.TypeMapping; -import org.apache.paimon.schema.Schema; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.types.DataField; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - -import org.apache.flink.configuration.Configuration; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; -import static org.apache.paimon.flink.action.cdc.kafka.KafkaActionUtils.getDataFormat; -import static org.apache.paimon.flink.action.cdc.kafka.KafkaActionUtils.getKafkaEarliestConsumer; -import static org.assertj.core.api.Assertions.assertThat; - -/** IT cases for {@link KafkaSyncTableAction}. */ -public class KafkaDebeziumWithSchemaSyncTableActionITCase extends KafkaActionITCaseBase { - - @Test - @Timeout(300) - public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution"); - } - - private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { - final String topic = "schema_evolution"; - createTestTopic(topic, 1, 1); - // ---------- Write the debezium json into Kafka ------------------- - List lines = - readLines( - String.format( - "kafka/debezium/table/schema/%s/debezium-data-1.txt", sourceDir)); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put("value.debezium-json.schema-include", "true"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topic, sourceDir); - } - - private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exception { - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format( - "kafka/debezium/table/schema/%s/debezium-data-2.txt", - sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", - "+I[102, car battery, 12V car battery, 8.1, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format( - "kafka/debezium/table/schema/%s/debezium-data-3.txt", - sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age", "address"}); - expected = - Arrays.asList( - "+I[102, car battery, 12V car battery, 8.1, NULL, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18, NULL]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24, NULL]", - "+I[105, hammer, 14oz carpenter's hammer, 0.875, NULL, Beijing]", - "+I[107, rocks, box of assorted rocks, 5.3, NULL, NULL]"); - waitForResult(expected, table, rowType, primaryKeys); - } - - @Test - @Timeout(60) - public void testComputedColumn() throws Exception { - String topic = "computed_column"; - createTestTopic(topic, 1, 1); - - List lines = - readLines("kafka/debezium/table/schema/computedcolumn/debezium-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.put("value.debezium-json.schema-include", "true"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withComputedColumnArgs("_year=year(date)") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), DataTypes.STRING(), DataTypes.INT() - }, - new String[] {"id", "date", "_year"}); - waitForResult( - Collections.singletonList("+I[101, 2023-03-23, 2023]"), - getFileStoreTable(tableName), - rowType, - Collections.singletonList("id")); - } - - @Test - @Timeout(60) - public void testKafkaBuildSchemaWithDelete() throws Exception { - final String topic = "test_kafka_schema"; - createTestTopic(topic, 1, 1); - // ---------- Write the Debezium json into Kafka ------------------- - List lines = - readLines("kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write debezium data to Kafka.", e); - } - Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); - kafkaConfig.setString(VALUE_FORMAT.key(), "debezium-json"); - kafkaConfig.setString(TOPIC.key(), topic); - - Schema kafkaSchema = - MessageQueueSchemaUtils.getSchema( - getKafkaEarliestConsumer(kafkaConfig), - getDataFormat(kafkaConfig), - TypeMapping.defaultMapping()); - List fields = new ArrayList<>(); - // {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14} - fields.add(new DataField(0, "id", DataTypes.STRING())); - fields.add(new DataField(1, "name", DataTypes.STRING())); - fields.add(new DataField(2, "description", DataTypes.STRING())); - fields.add(new DataField(3, "weight", DataTypes.STRING())); - assertThat(kafkaSchema.fields()).isEqualTo(fields); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java index 29ef6451d47c..189863f44c0d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncDatabaseActionITCase.java @@ -18,465 +18,58 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.testutils.assertj.AssertionUtils; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import javax.annotation.Nullable; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - /** IT cases for {@link KafkaSyncDatabaseAction}. */ -public class KafkaMaxwellSyncDatabaseActionITCase extends KafkaActionITCaseBase { +public class KafkaMaxwellSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { + + private static final String MAXWELL = "maxwell"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - final String topic1 = "schema_evolution_0"; - final String topic2 = "schema_evolution_1"; - boolean writeOne = false; - int fileCount = 2; - List topics = Arrays.asList(topic1, topic2); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the maxwell json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(i), - readLines( - "kafka/maxwell/database/schemaevolution/topic" - + i - + "/maxwell-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topics, writeOne, fileCount); + testSchemaEvolutionMultiTopic(MAXWELL); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - final String topic = "schema_evolution"; - boolean writeOne = true; - int fileCount = 2; - List topics = Collections.singletonList(topic); - topics.forEach(t -> createTestTopic(t, 1, 1)); - - // ---------- Write the maxwell json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(0), - readLines( - "kafka/maxwell/database/schemaevolution/topic" - + i - + "/maxwell-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topics, writeOne, fileCount); - } - - private void testSchemaEvolutionImpl(List topics, boolean writeOne, int fileCount) - throws Exception { - waitingTables("t1", "t2"); - - FileStoreTable table1 = getFileStoreTable("t1"); - FileStoreTable table2 = getFileStoreTable("t2"); - - RowType rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys1 = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - RowType rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys2 = Collections.singletonList("id"); - List expected2 = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected2, table2, rowType2, primaryKeys2); - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - writeOne ? topics.get(0) : topics.get(i), - readLines( - "kafka/maxwell/database/schemaevolution/topic" - + i - + "/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - - rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", - "+I[102, car battery, 12V car battery, 8.1, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "address"}); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, Beijing]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, Shanghai]"); - waitForResult(expected, table2, rowType2, primaryKeys2); + testSchemaEvolutionOneTopic(MAXWELL); } @Test public void testTopicIsEmpty() { - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - - KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); - - assertThatThrownBy(action::run) - .satisfies( - AssertionUtils.anyCauseMatches( - IllegalArgumentException.class, - "kafka_conf must and can only set one of the following options: topic,topic-pattern.")); + testTopicIsEmpty(MAXWELL); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - // create table t1 - createFileStoreTable( - "test_prefix_t1_test_suffix", - RowType.of( - new DataType[] { - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}), - Collections.emptyList(), - Collections.singletonList("id"), - Collections.emptyMap()); - - final String topic1 = "prefix_suffix_0"; - final String topic2 = "prefix_suffix_1"; - boolean writeOne = false; - int fileCount = 2; - List topics = Arrays.asList(topic1, topic2); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the maxwell json into Kafka ------------------- - - for (int i = 0; i < topics.size(); i++) { - try { - writeRecordsToKafka( - topics.get(i), - readLines( - "kafka/maxwell/database/prefixsuffix/topic" - + i - + "/maxwell-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTablePrefix("test_prefix_") - .withTableSuffix("_test_suffix") - .withTableConfig(getBasicTableConfig()) - // test including check with affix - .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") - .build(); - runActionWithDefaultEnv(action); - - testTableAffixImpl(topics, writeOne, fileCount); + testTableAffixMultiTopic(MAXWELL); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - // create table t1 - createFileStoreTable( - "test_prefix_t1_test_suffix", - RowType.of( - new DataType[] { - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}), - Collections.emptyList(), - Collections.singletonList("id"), - Collections.emptyMap()); - - final String topic1 = "prefix_suffix"; - List topics = Collections.singletonList(topic1); - boolean writeOne = true; - int fileCount = 2; - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the maxwell json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(0), - readLines( - "kafka/maxwell/database/prefixsuffix/topic" - + i - + "/maxwell-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTablePrefix("test_prefix_") - .withTableSuffix("_test_suffix") - .withTableConfig(getBasicTableConfig()) - // test including check with affix - .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") - .build(); - runActionWithDefaultEnv(action); - - testTableAffixImpl(topics, writeOne, fileCount); - } - - private void testTableAffixImpl(List topics, boolean writeOne, int fileCount) - throws Exception { - waitingTables("test_prefix_t1_test_suffix", "test_prefix_t2_test_suffix"); - - FileStoreTable table1 = getFileStoreTable("test_prefix_t1_test_suffix"); - FileStoreTable table2 = getFileStoreTable("test_prefix_t2_test_suffix"); - - RowType rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys1 = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - RowType rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys2 = Collections.singletonList("id"); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table2, rowType2, primaryKeys2); - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - writeOne ? topics.get(0) : topics.get(i), - readLines( - "kafka/maxwell/database/prefixsuffix/topic" - + i - + "/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - } - rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "address"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14, Beijing]", - "+I[102, car battery, 12V car battery, 8.1, Shanghai]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); - waitForResult(expected, table2, rowType2, primaryKeys2); + testTableAffixOneTopic(MAXWELL); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - includingAndExcludingTablesImpl( - "flink|paimon.+", - null, - Arrays.asList("flink", "paimon_1", "paimon_2"), - Collections.singletonList("ignore")); + testIncludingTables(MAXWELL); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - includingAndExcludingTablesImpl( - null, - "flink|paimon.+", - Collections.singletonList("ignore"), - Arrays.asList("flink", "paimon_1", "paimon_2")); + testExcludingTables(MAXWELL); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - includingAndExcludingTablesImpl( - "flink|paimon.+", - "paimon_1", - Arrays.asList("flink", "paimon_2"), - Arrays.asList("paimon_1", "ignore")); - } - - private void includingAndExcludingTablesImpl( - @Nullable String includingTables, - @Nullable String excludingTables, - List existedTables, - List notExistedTables) - throws Exception { - final String topic1 = "include_exclude" + UUID.randomUUID(); - List topics = Collections.singletonList(topic1); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the maxwell json into Kafka ------------------- - - try { - writeRecordsToKafka( - topics.get(0), - readLines("kafka/maxwell/database/include/topic0/maxwell-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .includingTables(includingTables) - .excludingTables(excludingTables) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - // check paimon tables - waitingTables(existedTables); - assertTableNotExists(notExistedTables); + testIncludingAndExcludingTables(MAXWELL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java index 856f66a1ecce..8d100111c191 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaMaxwellSyncTableActionITCase.java @@ -18,524 +18,71 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.table.AbstractFileStoreTable; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.EARLIEST_OFFSET; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.GROUP_OFFSETS; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.LATEST_OFFSET; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.SPECIFIC_OFFSETS; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.TIMESTAMP; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; -import static org.apache.paimon.testutils.assertj.AssertionUtils.anyCauseMatches; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaMaxwellSyncTableActionITCase extends KafkaSyncTableActionITCase { -/** IT cases for {@link KafkaMaxwellSyncTableActionITCase}. */ -public class KafkaMaxwellSyncTableActionITCase extends KafkaActionITCaseBase { + private static final String MAXWELL = "maxwell"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution"); - } - - private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { - final String topic = "schema_evolution"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = - readLines(String.format("kafka/maxwell/table/%s/maxwell-data-1.txt", sourceDir)); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topic, sourceDir); - } - - private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exception { - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format("kafka/maxwell/table/%s/maxwell-data-2.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", - "+I[102, car battery, 12V car battery, 8.1, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines( - String.format("kafka/maxwell/table/%s/maxwell-data-3.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age", "address"}); - expected = - Arrays.asList( - "+I[102, car battery, 12V car battery, 8.1, NULL, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18, NULL]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24, NULL]", - "+I[105, hammer, 14oz carpenter's hammer, 0.875, NULL, Beijing]", - "+I[107, rocks, box of assorted rocks, 5.3, NULL, NULL]"); - waitForResult(expected, table, rowType, primaryKeys); + runSingleTableSchemaEvolution("schemaevolution", MAXWELL); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - final String topic = "not_support"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/schemaevolution/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "togg-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - - assertThatThrownBy(action::run) - .satisfies( - anyCauseMatches( - UnsupportedOperationException.class, - "This format: togg-json is not supported.")); + testNotSupportFormat(MAXWELL); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - final String topic = "assert_schema_compatible"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/schemaevolution/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - - // create an incompatible table - createFileStoreTable( - RowType.of( - new DataType[] {DataTypes.STRING(), DataTypes.STRING()}, - new String[] {"k", "v1"}), - Collections.emptyList(), - Collections.singletonList("k"), - Collections.emptyMap()); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - - assertThatThrownBy(action::run) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Paimon schema and source table schema are not compatible.\n" - + "Paimon fields are: [`k` STRING NOT NULL, `v1` STRING].\n" - + "Source table fields are: [`id` STRING NOT NULL, `name` STRING, `description` STRING, `weight` STRING]")); + testAssertSchemaCompatible(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - final String topic = "start_up_specific"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/startupmode/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); - kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has two records we read two - List expected = - Collections.singletonList("+I[102, car battery, 12V car battery, 8.1]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionSpecific(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - final String topic = "start_up_latest"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/startupmode/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - Thread.sleep(5000); - FileStoreTable table = getFileStoreTable(tableName); - try { - writeRecordsToKafka( - topic, readLines("kafka/maxwell/table/startupmode/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read two - List expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionLatest(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - final String topic = "start_up_timestamp"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/startupmode/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); - kafkaConfig.put( - SCAN_STARTUP_TIMESTAMP_MILLIS.key(), String.valueOf(System.currentTimeMillis())); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka( - topic, readLines("kafka/maxwell/table/startupmode/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read two - List expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionTimestamp(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - final String topic = "start_up_earliest"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/startupmode/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka( - topic, readLines("kafka/maxwell/table/startupmode/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read all - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionEarliest(MAXWELL); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - final String topic = "start_up_group"; - createTestTopic(topic, 1, 1); - // ---------- Write the maxwell json into Kafka ------------------- - List lines = readLines("kafka/maxwell/table/startupmode/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka( - topic, readLines("kafka/maxwell/table/startupmode/maxwell-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write maxwell data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read all - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionGroup(MAXWELL); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - String topic = "computed_column"; - createTestTopic(topic, 1, 1); - - List lines = readLines("kafka/maxwell/table/computedcolumn/maxwell-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write canal data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPartitionKeys("_year") - .withPrimaryKeys("_id", "_year") - .withComputedColumnArgs("_year=year(_date)") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.INT().notNull() - }, - new String[] {"_id", "_date", "_year"}); - waitForResult( - Collections.singletonList("+I[101, 2023-03-23, 2023]"), - getFileStoreTable(tableName), - rowType, - Arrays.asList("_id", "_year")); + testComputedColumn(MAXWELL); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - String topic = "watermark"; - createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, readLines("kafka/maxwell/table/watermark/maxwell-data-1.txt")); - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "maxwell-json"); - kafkaConfig.put(TOPIC.key(), topic); - - Map config = getBasicTableConfig(); - config.put("tag.automatic-creation", "watermark"); - config.put("tag.creation-period", "hourly"); - config.put("scan.watermark.alignment.group", "alignment-group-1"); - config.put("scan.watermark.alignment.max-drift", "20 s"); - config.put("scan.watermark.alignment.update-interval", "1 s"); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); - runActionWithDefaultEnv(action); - - AbstractFileStoreTable table = - (AbstractFileStoreTable) catalog.getTable(new Identifier(database, tableName)); - while (true) { - if (table.snapshotManager().snapshotCount() > 0 - && table.snapshotManager().latestSnapshot().watermark() - != -9223372036854775808L) { - return; - } - Thread.sleep(1000); - } + testWaterMarkSyncTable(MAXWELL); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java index 07ea1e89f92b..9eacbfea2874 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncDatabaseActionITCase.java @@ -18,498 +18,63 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.FileSystemCatalogOptions; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.testutils.assertj.AssertionUtils; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import javax.annotation.Nullable; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - /** IT cases for {@link KafkaSyncDatabaseAction}. */ -public class KafkaOggSyncDatabaseActionITCase extends KafkaActionITCaseBase { +public class KafkaOggSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { + private static final String OGG = "ogg"; @Test @Timeout(60) public void testSchemaEvolutionMultiTopic() throws Exception { - final String topic1 = "schema_evolution_0"; - final String topic2 = "schema_evolution_1"; - boolean writeOne = false; - int fileCount = 2; - List topics = Arrays.asList(topic1, topic2); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the ogg json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(i), - readLines( - "kafka/ogg/database/schemaevolution/topic" - + i - + "/ogg-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topics, writeOne, fileCount); + testSchemaEvolutionMultiTopic(OGG); } @Test @Timeout(60) public void testSchemaEvolutionOneTopic() throws Exception { - final String topic = "schema_evolution"; - boolean writeOne = true; - int fileCount = 2; - List topics = Collections.singletonList(topic); - topics.forEach(t -> createTestTopic(t, 1, 1)); - - // ---------- Write the ogg json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(0), - readLines( - "kafka/ogg/database/schemaevolution/topic" - + i - + "/ogg-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topics, writeOne, fileCount); - } - - private void testSchemaEvolutionImpl(List topics, boolean writeOne, int fileCount) - throws Exception { - waitingTables("T1", "T2"); - - FileStoreTable table1 = getFileStoreTable("T1"); - FileStoreTable table2 = getFileStoreTable("T2"); - - RowType rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys1 = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - RowType rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys2 = Collections.singletonList("id"); - List expected2 = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected2, table2, rowType2, primaryKeys2); - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - writeOne ? topics.get(0) : topics.get(i), - readLines( - "kafka/ogg/database/schemaevolution/topic" - + i - + "/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - - rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175, NULL]", - "+I[102, car battery, 12V car battery, 8.100000381469727, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929, 19]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "address"}); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929, Beijing]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, Shanghai]"); - waitForResult(expected, table2, rowType2, primaryKeys2); + testSchemaEvolutionOneTopic(OGG); } @Test public void testTopicIsEmpty() { - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - - KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); - - assertThatThrownBy(action::run) - .satisfies( - AssertionUtils.anyCauseMatches( - IllegalArgumentException.class, - "kafka_conf must and can only set one of the following options: topic,topic-pattern.")); + testTopicIsEmpty(OGG); } @Test @Timeout(60) public void testTableAffixMultiTopic() throws Exception { - // create table t1 - createFileStoreTable( - "TEST_PREFIX_T1_TEST_SUFFIX", - RowType.of( - new DataType[] { - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}), - Collections.emptyList(), - Collections.singletonList("id"), - Collections.emptyMap()); - - final String topic1 = "prefix_suffix_0"; - final String topic2 = "prefix_suffix_1"; - boolean writeOne = false; - int fileCount = 2; - List topics = Arrays.asList(topic1, topic2); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the ogg json into Kafka ------------------- - - for (int i = 0; i < topics.size(); i++) { - try { - writeRecordsToKafka( - topics.get(i), - readLines("kafka/ogg/database/prefixsuffix/topic" + i + "/ogg-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTablePrefix("TEST_PREFIX_") - .withTableSuffix("_TEST_SUFFIX") - .withTableConfig(getBasicTableConfig()) - // test including check with affix - .includingTables(ThreadLocalRandom.current().nextBoolean() ? "T1|T2" : ".*") - .build(); - runActionWithDefaultEnv(action); - - testTableAffixImpl(topics, writeOne, fileCount); + testTableAffixMultiTopic(OGG); } @Test @Timeout(60) public void testTableAffixOneTopic() throws Exception { - // create table t1 - createFileStoreTable( - "TEST_PREFIX_T1_TEST_SUFFIX", - RowType.of( - new DataType[] { - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}), - Collections.emptyList(), - Collections.singletonList("id"), - Collections.emptyMap()); - - final String topic1 = "prefix_suffix"; - List topics = Collections.singletonList(topic1); - boolean writeOne = true; - int fileCount = 2; - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the ogg json into Kafka ------------------- - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - topics.get(0), - readLines("kafka/ogg/database/prefixsuffix/topic" + i + "/ogg-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTablePrefix("TEST_PREFIX_") - .withTableSuffix("_TEST_SUFFIX") - .withTableConfig(getBasicTableConfig()) - // test including check with affix - .includingTables(ThreadLocalRandom.current().nextBoolean() ? "T1|T2" : ".*") - .build(); - runActionWithDefaultEnv(action); - - testTableAffixImpl(topics, writeOne, fileCount); - } - - private void testTableAffixImpl(List topics, boolean writeOne, int fileCount) - throws Exception { - waitingTables("TEST_PREFIX_T1_TEST_SUFFIX", "TEST_PREFIX_T2_TEST_SUFFIX"); - - FileStoreTable table1 = getFileStoreTable("TEST_PREFIX_T1_TEST_SUFFIX"); - FileStoreTable table2 = getFileStoreTable("TEST_PREFIX_T2_TEST_SUFFIX"); - - RowType rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys1 = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - RowType rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys2 = Collections.singletonList("id"); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table2, rowType2, primaryKeys2); - - for (int i = 0; i < fileCount; i++) { - try { - writeRecordsToKafka( - writeOne ? topics.get(0) : topics.get(i), - readLines("kafka/ogg/database/prefixsuffix/topic" + i + "/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - } - rowType1 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "address"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175, Beijing]", - "+I[102, car battery, 12V car battery, 8.100000381469727, Shanghai]"); - waitForResult(expected, table1, rowType1, primaryKeys1); - - rowType2 = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929, 19]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); - waitForResult(expected, table2, rowType2, primaryKeys2); + testTableAffixOneTopic(OGG); } @Test @Timeout(60) public void testIncludingTables() throws Exception { - includingAndExcludingTablesImpl( - "FLINK|PAIMON.+", - null, - Arrays.asList("FLINK", "PAIMON_1", "PAIMON_2"), - Collections.singletonList("IGNORE")); + testIncludingTables(OGG); } @Test @Timeout(60) public void testExcludingTables() throws Exception { - includingAndExcludingTablesImpl( - null, - "FLINK|PAIMON.+", - Collections.singletonList("IGNORE"), - Arrays.asList("FLINK", "PAIMON_1", "PAIMON_2")); + testExcludingTables(OGG); } @Test @Timeout(60) public void testIncludingAndExcludingTables() throws Exception { - includingAndExcludingTablesImpl( - "FLINK|PAIMON.+", - "PAIMON_1", - Arrays.asList("FLINK", "PAIMON_2"), - Arrays.asList("PAIMON_1", "IGNORE")); - } - - private void includingAndExcludingTablesImpl( - @Nullable String includingTables, - @Nullable String excludingTables, - List existedTables, - List notExistedTables) - throws Exception { - final String topic1 = "include_exclude" + UUID.randomUUID(); - List topics = Collections.singletonList(topic1); - topics.forEach(topic -> createTestTopic(topic, 1, 1)); - - // ---------- Write the ogg json into Kafka ------------------- - - try { - writeRecordsToKafka( - topics.get(0), readLines("kafka/ogg/database/include/topic0/ogg-data-1.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - // try synchronization - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), String.join(";", topics)); - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .includingTables(includingTables) - .excludingTables(excludingTables) - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - // check paimon tables - waitingTables(existedTables); - assertTableNotExists(notExistedTables); + testIncludingAndExcludingTables(OGG); } @Test @Timeout(60) public void testCaseInsensitive() throws Exception { - final String topic = "case-insensitive"; - createTestTopic(topic, 1, 1); - - // ---------- Write the ogg json into Kafka ------------------- - - writeRecordsToKafka(topic, readLines("kafka/ogg/database/case-insensitive/ogg-data-1.txt")); - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - - KafkaSyncDatabaseAction action = - syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .withCatalogConfig( - Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) - .build(); - runActionWithDefaultEnv(action); - - waitingTables("t1"); - FileStoreTable table = getFileStoreTable("t1"); - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys1 = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expected, table, rowType, primaryKeys1); + testCaseInsensitive(OGG); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java index 1c44970217f3..6394b5b71978 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaOggSyncTableActionITCase.java @@ -18,580 +18,77 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.table.AbstractFileStoreTable; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; -import static org.apache.paimon.testutils.assertj.AssertionUtils.anyCauseMatches; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaOggSyncTableActionITCase extends KafkaSyncTableActionITCase { -/** IT cases for {@link KafkaOggSyncTableActionITCase}. */ -public class KafkaOggSyncTableActionITCase extends KafkaActionITCaseBase { + private static final String OGG = "ogg"; @Test @Timeout(60) public void testSchemaEvolution() throws Exception { - runSingleTableSchemaEvolution("schemaevolution"); - } - - private void runSingleTableSchemaEvolution(String sourceDir) throws Exception { - final String topic = "schema_evolution"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = - readLines(String.format("kafka/ogg/table/%s/ogg-data-1.txt", sourceDir)); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - testSchemaEvolutionImpl(topic, sourceDir); - } - - private void testSchemaEvolutionImpl(String topic, String sourceDir) throws Exception { - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines(String.format("kafka/ogg/table/%s/ogg-data-2.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age"}); - expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175, NULL]", - "+I[102, car battery, 12V car battery, 8.100000381469727, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929, 18]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); - waitForResult(expected, table, rowType, primaryKeys); - - try { - writeRecordsToKafka( - topic, - readLines(String.format("kafka/ogg/table/%s/ogg-data-3.txt", sourceDir))); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight", "age", "address"}); - expected = - Arrays.asList( - "+I[102, car battery, 12V car battery, 8.100000381469727, NULL, NULL]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929, 18, NULL]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24, NULL]", - "+I[105, hammer, 14oz carpenter's hammer, 0.875, NULL, Beijing]", - "+I[107, rocks, box of assorted rocks, 5.300000190734863, NULL, NULL]"); - waitForResult(expected, table, rowType, primaryKeys); + runSingleTableSchemaEvolution("schemaevolution", OGG); } @Test @Timeout(60) public void testNotSupportFormat() throws Exception { - final String topic = "not_support"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/schemaevolution/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "togg-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - - assertThatThrownBy(action::run) - .satisfies( - anyCauseMatches( - UnsupportedOperationException.class, - "This format: togg-json is not supported.")); + testNotSupportFormat(OGG); } @Test @Timeout(60) public void testAssertSchemaCompatible() throws Exception { - final String topic = "assert_schema_compatible"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/schemaevolution/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - - // create an incompatible table - createFileStoreTable( - RowType.of( - new DataType[] {DataTypes.STRING(), DataTypes.STRING()}, - new String[] {"k", "v1"}), - Collections.emptyList(), - Collections.singletonList("k"), - Collections.emptyMap()); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - - assertThatThrownBy(action::run) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Paimon schema and source table schema are not compatible.\n" - + "Paimon fields are: [`k` STRING NOT NULL, `v1` STRING].\n" - + "Source table fields are: [`id` STRING NOT NULL, `name` STRING, `description` STRING, `weight` STRING]")); + testAssertSchemaCompatible(OGG); } @Test @Timeout(60) public void testStarUpOptionSpecific() throws Exception { - final String topic = "start_up_specific"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/startupmode/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put("scan.startup.mode", "specific-offsets"); - kafkaConfig.put("scan.startup.specific-offsets", "partition:0,offset:1"); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has two records we read two - List expected = - Collections.singletonList( - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionSpecific(OGG); } @Test @Timeout(60) public void testStarUpOptionLatest() throws Exception { - final String topic = "start_up_latest"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/startupmode/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put("scan.startup.mode", "latest-offset"); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - Thread.sleep(5000); - FileStoreTable table = getFileStoreTable(tableName); - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/startupmode/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read two - List expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionLatest(OGG); } @Test @Timeout(60) public void testStarUpOptionTimestamp() throws Exception { - final String topic = "start_up_timestamp"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/startupmode/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put("scan.startup.mode", "timestamp"); - kafkaConfig.put( - "scan.startup.timestamp-millis", String.valueOf(System.currentTimeMillis())); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/startupmode/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read two - List expected = - Arrays.asList( - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionTimestamp(OGG); } @Test @Timeout(60) public void testStarUpOptionEarliest() throws Exception { - final String topic = "start_up_earliest"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/startupmode/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put("scan.startup.mode", "earliest-offset"); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/startupmode/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read all - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionEarliest(OGG); } @Test @Timeout(60) public void testStarUpOptionGroup() throws Exception { - final String topic = "start_up_group"; - createTestTopic(topic, 1, 1); - // ---------- Write the ogg json into Kafka ------------------- - List lines = readLines("kafka/ogg/table/startupmode/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - kafkaConfig.put("scan.startup.mode", "group-offsets"); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/startupmode/ogg-data-2.txt")); - } catch (Exception e) { - throw new Exception("Failed to write ogg data to Kafka.", e); - } - FileStoreTable table = getFileStoreTable(tableName); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - List primaryKeys = Collections.singletonList("id"); - // topic has four records we read all - List expected = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800000011920929]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); - waitForResult(expected, table, rowType, primaryKeys); + testStarUpOptionGroup(OGG); } @Test @Timeout(60) public void testComputedColumn() throws Exception { - String topic = "computed_column"; - createTestTopic(topic, 1, 1); - - List lines = readLines("kafka/ogg/table/computedcolumn/ogg-data-1.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write canal data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPartitionKeys("_year") - .withPrimaryKeys("_id", "_year") - .withComputedColumnArgs("_year=year(_date)") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.INT().notNull() - }, - new String[] {"_id", "_date", "_year"}); - waitForResult( - Collections.singletonList("+I[101, 2023-03-23, 2023]"), - getFileStoreTable(tableName), - rowType, - Arrays.asList("_id", "_year")); + testComputedColumn(OGG); } @Test @Timeout(60) public void testCDCOperations() throws Exception { - String topic = "event"; - createTestTopic(topic, 1, 1); - - List lines = readLines("kafka/ogg/table/event/event-insert.txt"); - try { - writeRecordsToKafka(topic, lines); - } catch (Exception e) { - throw new Exception("Failed to write canal data to Kafka.", e); - } - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig) - .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) - .build(); - runActionWithDefaultEnv(action); - - FileStoreTable table = getFileStoreTable(tableName); - List primaryKeys = Collections.singletonList("id"); - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.STRING().notNull(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.STRING() - }, - new String[] {"id", "name", "description", "weight"}); - - // For the INSERT operation - List expectedInsert = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]", - "+I[103, scooter, Big 2-wheel scooter , 5.179999828338623]"); - waitForResult(expectedInsert, table, rowType, primaryKeys); - - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/event/event-update.txt")); - } catch (Exception e) { - throw new Exception("Failed to write canal data to Kafka.", e); - } - // For the UPDATE operation - List expectedUpdate = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]", - "+I[103, scooter, Big 2-wheel scooter , 8.170000076293945]"); - waitForResult(expectedUpdate, table, rowType, primaryKeys); - - try { - writeRecordsToKafka(topic, readLines("kafka/ogg/table/event/event-delete.txt")); - } catch (Exception e) { - throw new Exception("Failed to write canal data to Kafka.", e); - } - - // For the REPLACE operation - List expectedReplace = - Arrays.asList( - "+I[101, scooter, Small 2-wheel scooter, 3.140000104904175]", - "+I[102, car battery, 12V car battery, 8.100000381469727]"); - waitForResult(expectedReplace, table, rowType, primaryKeys); + testCDCOperations(OGG); } @Test @Timeout(60) public void testWaterMarkSyncTable() throws Exception { - String topic = "watermark"; - createTestTopic(topic, 1, 1); - writeRecordsToKafka(topic, readLines("kafka/ogg/table/watermark/ogg-data-1.txt")); - - Map kafkaConfig = getBasicKafkaConfig(); - kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); - kafkaConfig.put(TOPIC.key(), topic); - - Map config = getBasicTableConfig(); - config.put("tag.automatic-creation", "watermark"); - config.put("tag.creation-period", "hourly"); - config.put("scan.watermark.alignment.group", "alignment-group-1"); - config.put("scan.watermark.alignment.max-drift", "20 s"); - config.put("scan.watermark.alignment.update-interval", "1 s"); - - KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); - runActionWithDefaultEnv(action); - - AbstractFileStoreTable table = - (AbstractFileStoreTable) catalog.getTable(new Identifier(database, tableName)); - while (true) { - if (table.snapshotManager().snapshotCount() > 0 - && table.snapshotManager().latestSnapshot().watermark() - != -9223372036854775808L) { - return; - } - Thread.sleep(1000); - } + testWaterMarkSyncTable(OGG); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java new file mode 100644 index 000000000000..66e704927d25 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -0,0 +1,556 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.testutils.assertj.AssertionUtils; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT cases for {@link KafkaSyncDatabaseAction}. */ +public class KafkaSyncDatabaseActionITCase extends KafkaActionITCaseBase { + + protected void testSchemaEvolutionMultiTopic(String format) throws Exception { + final String topic1 = "schema_evolution_0"; + final String topic2 = "schema_evolution_1"; + boolean writeOne = false; + int fileCount = 2; + List topics = Arrays.asList(topic1, topic2); + topics.forEach(topic -> createTestTopic(topic, 1, 1)); + + // ---------- Write the data into Kafka ------------------- + + for (int i = 0; i < fileCount; i++) { + try { + writeRecordsToKafka( + topics.get(i), + readLines( + String.format( + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), String.join(";", topics)); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + testSchemaEvolutionImpl(topics, writeOne, fileCount, format); + } + + protected void testSchemaEvolutionOneTopic(String format) throws Exception { + final String topic = "schema_evolution"; + boolean writeOne = true; + int fileCount = 2; + List topics = Collections.singletonList(topic); + topics.forEach(t -> createTestTopic(t, 1, 1)); + + // ---------- Write the maxwell json into Kafka ------------------- + + for (int i = 0; i < fileCount; i++) { + try { + writeRecordsToKafka( + topics.get(0), + readLines( + String.format( + "kafka/%s/database/schemaevolution/topic%s/%s-data-1.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), String.join(";", topics)); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + testSchemaEvolutionImpl(topics, writeOne, fileCount, format); + } + + private void testSchemaEvolutionImpl( + List topics, boolean writeOne, int fileCount, String format) throws Exception { + waitingTables("t1", "t2"); + + FileStoreTable table1 = getFileStoreTable("t1"); + FileStoreTable table2 = getFileStoreTable("t2"); + + RowType rowType1 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expected, table1, rowType1, getPrimaryKey(format)); + + RowType rowType2 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + List expected2 = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected2, table2, rowType2, getPrimaryKey(format)); + + for (int i = 0; i < fileCount; i++) { + try { + writeRecordsToKafka( + writeOne ? topics.get(0) : topics.get(i), + readLines( + String.format( + "kafka/%s/database/schemaevolution/topic%s/%s-data-2.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + + rowType1 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "age"}); + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[102, car battery, 12V car battery, 8.1, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); + waitForResult(expected, table1, rowType1, getPrimaryKey(format)); + + rowType2 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "address"}); + + if (format.equals("debezium")) { + expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, Beijing]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, NULL]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, Shanghai]"); + } else { + expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, Beijing]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, Shanghai]"); + } + + waitForResult(expected, table2, rowType2, getPrimaryKey(format)); + } + + protected void testTopicIsEmpty(String format) { + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + + KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); + + assertThatThrownBy(action::run) + .satisfies( + AssertionUtils.anyCauseMatches( + IllegalArgumentException.class, + "kafka_conf must and can only set one of the following options: topic,topic-pattern.")); + } + + protected void testTableAffixMultiTopic(String format) throws Exception { + // create table t1 + createFileStoreTable( + "test_prefix_t1_test_suffix", + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}), + Collections.emptyList(), + getPrimaryKey(format), + Collections.emptyMap()); + + final String topic1 = "prefix_suffix_0"; + final String topic2 = "prefix_suffix_1"; + boolean writeOne = false; + int fileCount = 2; + List topics = Arrays.asList(topic1, topic2); + topics.forEach(topic -> createTestTopic(topic, 1, 1)); + + // ---------- Write the data into Kafka ------------------- + + for (int i = 0; i < topics.size(); i++) { + try { + writeRecordsToKafka( + topics.get(i), + readLines( + String.format( + "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + + // try synchronization + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), String.join(";", topics)); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTablePrefix("test_prefix_") + .withTableSuffix("_test_suffix") + .withTableConfig(getBasicTableConfig()) + // test including check with affix + .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") + .build(); + runActionWithDefaultEnv(action); + + testTableAffixImpl(topics, writeOne, fileCount, format); + } + + protected void testTableAffixOneTopic(String format) throws Exception { + // create table t1 + createFileStoreTable( + "test_prefix_t1_test_suffix", + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}), + Collections.emptyList(), + getPrimaryKey(format), + Collections.emptyMap()); + + final String topic1 = "prefix_suffix"; + List topics = Collections.singletonList(topic1); + boolean writeOne = true; + int fileCount = 2; + topics.forEach(topic -> createTestTopic(topic, 1, 1)); + + // ---------- Write the maxwell json into Kafka ------------------- + + for (int i = 0; i < fileCount; i++) { + try { + writeRecordsToKafka( + topics.get(0), + readLines( + String.format( + "kafka/%s/database/prefixsuffix/topic%s/%s-data-1.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + + // try synchronization + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), String.join(";", topics)); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTablePrefix("test_prefix_") + .withTableSuffix("_test_suffix") + .withTableConfig(getBasicTableConfig()) + // test including check with affix + .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") + .build(); + runActionWithDefaultEnv(action); + + testTableAffixImpl(topics, writeOne, fileCount, format); + } + + private void testTableAffixImpl( + List topics, boolean writeOne, int fileCount, String format) throws Exception { + waitingTables("test_prefix_t1_test_suffix", "test_prefix_t2_test_suffix"); + + FileStoreTable table1 = getFileStoreTable("test_prefix_t1_test_suffix"); + FileStoreTable table2 = getFileStoreTable("test_prefix_t2_test_suffix"); + + RowType rowType1 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expected, table1, rowType1, getPrimaryKey(format)); + + RowType rowType2 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected, table2, rowType2, getPrimaryKey(format)); + + for (int i = 0; i < fileCount; i++) { + try { + writeRecordsToKafka( + writeOne ? topics.get(0) : topics.get(i), + readLines( + String.format( + "kafka/%s/database/prefixsuffix/topic%s/%s-data-2.txt", + format, i, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + } + rowType1 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "address"}); + if (format.equals("debezium")) { + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, Beijing]", + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[102, car battery, 12V car battery, 8.1, Shanghai]", + "+I[102, car battery, 12V car battery, 8.1, NULL]"); + } else { + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, Beijing]", + "+I[102, car battery, 12V car battery, 8.1, Shanghai]"); + } + waitForResult(expected, table1, rowType1, getPrimaryKey(format)); + + rowType2 = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "age"}); + if (format.equals("debezium")) { + expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, NULL]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, NULL]"); + } else { + expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 19]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 25]"); + } + waitForResult(expected, table2, rowType2, getPrimaryKey(format)); + } + + protected void testIncludingTables(String format) throws Exception { + includingAndExcludingTablesImpl( + "flink|paimon.+", + null, + Arrays.asList("flink", "paimon_1", "paimon_2"), + Collections.singletonList("ignore"), + format); + } + + protected void testExcludingTables(String format) throws Exception { + includingAndExcludingTablesImpl( + null, + "flink|paimon.+", + Collections.singletonList("ignore"), + Arrays.asList("flink", "paimon_1", "paimon_2"), + format); + } + + protected void testIncludingAndExcludingTables(String format) throws Exception { + includingAndExcludingTablesImpl( + "flink|paimon.+", + "paimon_1", + Arrays.asList("flink", "paimon_2"), + Arrays.asList("paimon_1", "ignore"), + format); + } + + private void includingAndExcludingTablesImpl( + @Nullable String includingTables, + @Nullable String excludingTables, + List existedTables, + List notExistedTables, + String format) + throws Exception { + final String topic1 = "include_exclude" + UUID.randomUUID(); + List topics = Collections.singletonList(topic1); + topics.forEach(topic -> createTestTopic(topic, 1, 1)); + + // ---------- Write the data into Kafka ------------------- + + try { + writeRecordsToKafka( + topics.get(0), + readLines( + String.format( + "kafka/%s/database/include/topic0/%s-data-1.txt", + format, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + // try synchronization + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), String.join(";", topics)); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .includingTables(includingTables) + .excludingTables(excludingTables) + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + // check paimon tables + waitingTables(existedTables); + assertTableNotExists(notExistedTables); + } + + protected void testCaseInsensitive(String format) throws Exception { + final String topic = "case-insensitive"; + createTestTopic(topic, 1, 1); + + // ---------- Write the data into Kafka ------------------- + + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/database/case-insensitive/%s-data-1.txt", + format, format))); + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTableConfig(getBasicTableConfig()) + .withCatalogConfig( + Collections.singletonMap( + FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + .build(); + runActionWithDefaultEnv(action); + + waitingTables("t1"); + FileStoreTable table = getFileStoreTable("t1"); + RowType rowType = + RowType.of( + new DataType[] { + getDataType(format), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expected, table, rowType, getPrimaryKey(format)); + } + + private DataType getDataType(String format) { + return format.equals("debezium") ? DataTypes.STRING() : DataTypes.STRING().notNull(); + } + + private List getPrimaryKey(String format) { + return format.equals("debezium") + ? Collections.emptyList() + : Collections.singletonList("id"); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java new file mode 100644 index 000000000000..f025e9c96207 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -0,0 +1,666 @@ +/* + * 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.paimon.flink.action.cdc.kafka; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.AbstractFileStoreTable; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.flink.configuration.Configuration; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.EARLIEST_OFFSET; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.GROUP_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.LATEST_OFFSET; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode.TIMESTAMP; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.apache.paimon.flink.action.cdc.kafka.KafkaActionUtils.getDataFormat; +import static org.apache.paimon.flink.action.cdc.kafka.KafkaActionUtils.getKafkaEarliestConsumer; +import static org.apache.paimon.testutils.assertj.AssertionUtils.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaSyncTableActionITCase extends KafkaActionITCaseBase { + + protected void runSingleTableSchemaEvolution(String sourceDir, String format) throws Exception { + final String topic = "schema_evolution"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format( + "kafka/%s/table/%s/%s-data-1.txt", format, sourceDir, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + testSchemaEvolutionImpl(topic, sourceDir, format); + } + + private void testSchemaEvolutionImpl(String topic, String sourceDir, String format) + throws Exception { + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expected, table, rowType, primaryKeys); + + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/%s/%s-data-2.txt", format, sourceDir, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "age"}); + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[102, car battery, 12V car battery, 8.1, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); + waitForResult(expected, table, rowType, primaryKeys); + + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/%s/%s-data-3.txt", format, sourceDir, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight", "age", "address"}); + expected = + Arrays.asList( + "+I[102, car battery, 12V car battery, 8.1, NULL, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18, NULL]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24, NULL]", + "+I[105, hammer, 14oz carpenter's hammer, 0.875, NULL, Beijing]", + "+I[107, rocks, box of assorted rocks, 5.3, NULL, NULL]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + public void testNotSupportFormat(String format) throws Exception { + final String topic = "not_support"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format( + "kafka/%s/table/schemaevolution/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), "togg-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + UnsupportedOperationException.class, + "This format: togg-json is not supported.")); + } + + protected void testAssertSchemaCompatible(String format) throws Exception { + final String topic = "assert_schema_compatible"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format( + "kafka/%s/table/schemaevolution/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + + // create an incompatible table + createFileStoreTable( + RowType.of( + new DataType[] {DataTypes.STRING(), DataTypes.STRING()}, + new String[] {"k", "v1"}), + Collections.emptyList(), + Collections.singletonList("k"), + Collections.emptyMap()); + + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Paimon schema and source table schema are not compatible.\n" + + "Paimon fields are: [`k` STRING NOT NULL, `v1` STRING].\n" + + "Source table fields are: [`id` STRING NOT NULL, `name` STRING, `description` STRING, `weight` STRING]")); + } + + protected void testStarUpOptionSpecific(String format) throws Exception { + final String topic = "start_up_specific"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format("kafka/%s/table/startupmode/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + kafkaConfig.put(SCAN_STARTUP_MODE.key(), SPECIFIC_OFFSETS.toString()); + kafkaConfig.put(SCAN_STARTUP_SPECIFIC_OFFSETS.key(), "partition:0,offset:1"); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + // topic has two records we read two + List expected = + Collections.singletonList("+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + protected void testStarUpOptionLatest(String format) throws Exception { + final String topic = "start_up_latest"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format("kafka/%s/table/startupmode/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + kafkaConfig.put(SCAN_STARTUP_MODE.key(), LATEST_OFFSET.toString()); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + Thread.sleep(5000); + FileStoreTable table = getFileStoreTable(tableName); + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/startupmode/%s-data-2.txt", format, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + // topic has four records we read two + List expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + public void testStarUpOptionTimestamp(String format) throws Exception { + final String topic = "start_up_timestamp"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format("kafka/%s/table/startupmode/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + kafkaConfig.put(SCAN_STARTUP_MODE.key(), TIMESTAMP.toString()); + kafkaConfig.put( + SCAN_STARTUP_TIMESTAMP_MILLIS.key(), String.valueOf(System.currentTimeMillis())); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/startupmode/%s-data-2.txt", format, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + // topic has four records we read two + List expected = + Arrays.asList( + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + public void testStarUpOptionEarliest(String format) throws Exception { + final String topic = "start_up_earliest"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format("kafka/%s/table/startupmode/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + kafkaConfig.put(SCAN_STARTUP_MODE.key(), EARLIEST_OFFSET.toString()); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/startupmode/%s-data-2.txt", format, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + // topic has four records we read all + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + public void testStarUpOptionGroup(String format) throws Exception { + final String topic = "start_up_group"; + createTestTopic(topic, 1, 1); + // ---------- Write the data into Kafka ------------------- + List lines = + readLines( + String.format("kafka/%s/table/startupmode/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + kafkaConfig.put(SCAN_STARTUP_MODE.key(), GROUP_OFFSETS.toString()); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + try { + writeRecordsToKafka( + topic, + readLines( + String.format( + "kafka/%s/table/startupmode/%s-data-2.txt", format, format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + // topic has four records we read all + List expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + public void testComputedColumn(String format) throws Exception { + String topic = "computed_column"; + createTestTopic(topic, 1, 1); + + List lines = + readLines( + String.format( + "kafka/%s/table/computedcolumn/%s-data-1.txt", format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPartitionKeys("_year") + .withPrimaryKeys("_id", "_year") + .withComputedColumnArgs("_year=year(_date)") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.INT().notNull() + }, + new String[] {"_id", "_date", "_year"}); + waitForResult( + Collections.singletonList("+I[101, 2023-03-23, 2023]"), + getFileStoreTable(tableName), + rowType, + Arrays.asList("_id", "_year")); + } + + protected void testCDCOperations(String format) throws Exception { + String topic = "event"; + createTestTopic(topic, 1, 1); + + List lines = + readLines(String.format("kafka/%s/table/event/event-insert.txt", format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), "ogg-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(tableName); + List primaryKeys = Collections.singletonList("id"); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.STRING().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING() + }, + new String[] {"id", "name", "description", "weight"}); + + // For the INSERT operation + List expectedInsert = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]", + "+I[103, scooter, Big 2-wheel scooter , 5.1]"); + waitForResult(expectedInsert, table, rowType, primaryKeys); + + try { + writeRecordsToKafka( + topic, + readLines(String.format("kafka/%s/table/event/event-update.txt", format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + // For the UPDATE operation + List expectedUpdate = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]", + "+I[103, scooter, Big 2-wheel scooter , 8.1]"); + waitForResult(expectedUpdate, table, rowType, primaryKeys); + + try { + writeRecordsToKafka( + topic, + readLines(String.format("kafka/%s/table/event/event-delete.txt", format))); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + + // For the REPLACE operation + List expectedReplace = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]"); + waitForResult(expectedReplace, table, rowType, primaryKeys); + } + + public void testKafkaBuildSchemaWithDelete(String format) throws Exception { + final String topic = "test_kafka_schema"; + createTestTopic(topic, 1, 1); + // ---------- Write the Debezium json into Kafka ------------------- + List lines = + readLines( + String.format( + "kafka/%s/table/schema/schemaevolution/%s-data-4.txt", + format, format)); + try { + writeRecordsToKafka(topic, lines); + } catch (Exception e) { + throw new Exception(String.format("Failed to write %s data to Kafka.", format), e); + } + Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); + kafkaConfig.setString(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.setString(TOPIC.key(), topic); + + Schema kafkaSchema = + MessageQueueSchemaUtils.getSchema( + getKafkaEarliestConsumer(kafkaConfig), + getDataFormat(kafkaConfig), + TypeMapping.defaultMapping()); + List fields = new ArrayList<>(); + // {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14} + fields.add(new DataField(0, "id", DataTypes.STRING())); + fields.add(new DataField(1, "name", DataTypes.STRING())); + fields.add(new DataField(2, "description", DataTypes.STRING())); + fields.add(new DataField(3, "weight", DataTypes.STRING())); + assertThat(kafkaSchema.fields()).isEqualTo(fields); + } + + public void testWaterMarkSyncTable(String format) throws Exception { + String topic = "watermark"; + createTestTopic(topic, 1, 1); + writeRecordsToKafka( + topic, + readLines(String.format("kafka/%s/table/watermark/%s-data-1.txt", format, format))); + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + + Map config = getBasicTableConfig(); + config.put("tag.automatic-creation", "watermark"); + config.put("tag.creation-period", "hourly"); + config.put("scan.watermark.alignment.group", "alignment-group-1"); + config.put("scan.watermark.alignment.max-drift", "20 s"); + config.put("scan.watermark.alignment.update-interval", "1 s"); + + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); + runActionWithDefaultEnv(action); + + AbstractFileStoreTable table = + (AbstractFileStoreTable) catalog.getTable(new Identifier(database, tableName)); + while (true) { + if (table.snapshotManager().snapshotCount() > 0 + && table.snapshotManager().latestSnapshot().watermark() + != -9223372036854775808L) { + return; + } + Thread.sleep(1000); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt new file mode 100644 index 000000000000..968b66cc8574 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/include/topic0/debezium-data-1.txt @@ -0,0 +1,22 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "paimon_1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "paimon_2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "ignore", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "flink", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt new file mode 100644 index 000000000000..0b0181c43180 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-1.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt new file mode 100644 index 000000000000..7ab996b7841c --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic0/debezium-data-2.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14, "address":"Beijing"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1, "address":"Shanghai"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt new file mode 100644 index 000000000000..5893466ed304 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-1.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt new file mode 100644 index 000000000000..3a0d84b1df40 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/prefixsuffix/topic1/debezium-data-2.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8, "age":19}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "age":25}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt new file mode 100644 index 000000000000..0b0181c43180 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-1.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt new file mode 100644 index 000000000000..e1ce0ddf9030 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic0/debezium-data-2.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8, "age": 19}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "age": 25}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t1", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt new file mode 100644 index 000000000000..5893466ed304 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-1.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt new file mode 100644 index 000000000000..f14db607f118 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/database/schemaevolution/topic1/debezium-data-2.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8, "address":"Beijing"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "address":"Shanghai"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "t2", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt index 5a571d3fd383..fce341e17d4d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/computedcolumn/debezium-data-1.txt @@ -16,4 +16,4 @@ * limitations under the License. */ -{"before": null, "after": {"id": 101, "date": "2023-03-23"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"_id":101,"_date":"2023-03-23"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt new file mode 100644 index 000000000000..b3ff4e23a3fb --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-1.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt new file mode 100644 index 000000000000..e2001b81f8c7 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/startupmode/debezium-data-2.txt @@ -0,0 +1,20 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} +{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt index 957a40d8fc10..cc20491d6ef4 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/case-insensitive/ogg-data-1.txt @@ -17,5 +17,5 @@ */ -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000143","primary_keys":["ID"],"after":{"ID":101,"NAME":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"ID":102,"name":"car battery","description":"12V car battery","WEIGHT":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file +{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000143","primary_keys":["ID"],"after":{"ID":101,"NAME":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"ID":102,"name":"car battery","description":"12V car battery","WEIGHT":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt index eca34df44793..41ce31021999 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/include/topic0/ogg-data-1.txt @@ -16,7 +16,7 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE_AFFIX.PAIMON_1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.PAIMON_2","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.IGNORE","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.FLINK","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.paimon_1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.paimon_2","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.ignore","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.flink","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt index 21f53284c278..066c6cf8c468 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt index c904ab909b4f..5856100b3883 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic0/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175,"address":"Beijing"},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727,"address":"Shanghai"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14,"address":"Beijing"},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1,"address":"Shanghai"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt index ea1dc2d9a90a..55ec1f37ec2f 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE_AFFIX.T2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE_AFFIX.t2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt index 68f47d8e3e16..c41a8033b538 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/prefixsuffix/topic1/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929,"age":19},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":19},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt index 545481ef31b4..471779724b37 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file +{"table":"PAIMON_SYNC_DATABASE.t1","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"PAIMON_SYNC_DATABASE.t1","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt index d9f8f37e453f..6bc57cc3b2f1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic0/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929,"age":19},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE.T1","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t1","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":19},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t1","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt index f351beafc21c..809de4743651 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt index c4f5f5e41703..62b30b32ada8 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/database/schemaevolution/topic1/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929,"address":"Beijing"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"PAIMON_SYNC_DATABASE.T2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"address":"Shanghai"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"address":"Beijing"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"PAIMON_SYNC_DATABASE.t2","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"address":"Shanghai"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt index c6f3b831af2e..59b10240f7e8 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-delete.txt @@ -16,4 +16,4 @@ * limitations under the License. */ -{"table":"OGG.EVENT","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":8.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} +{"table":"OGG.EVENT","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":8.1},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt index b1eb89aed519..e391af96cc73 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-insert.txt @@ -16,8 +16,8 @@ * limitations under the License. */ -{"table":"OGG.EVENT","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"OGG.EVENT","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.EVENT","pos":"00000000000000000000155","primary_keys":["id"],"after":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} +{"table":"OGG.EVENT","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"OGG.EVENT","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.EVENT","pos":"00000000000000000000155","primary_keys":["id"],"after":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.1},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt index 91c047b1c9f5..eef1927244f1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/event/event-update.txt @@ -16,4 +16,4 @@ * limitations under the License. */ -{"table":"OGG.EVENT","pos":"00000000000000000000157","primary_keys":["id"],"before":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":8.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} +{"table":"OGG.EVENT","pos":"00000000000000000000157","primary_keys":["id"],"before":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.1},"after":{"id":103,"name":"scooter","description":"Big 2-wheel scooter ","weight":8.1},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt index 5b6ef10854db..adeabb850ffb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt index efa781c7a197..dc9f6121a497 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929,"age":18},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":18},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} {"table":"OGG.TBL_TEST","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":24},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt index 0e76c17de35e..62feca44f039 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/schemaevolution/ogg-data-3.txt @@ -17,7 +17,7 @@ */ {"table":"OGG.TBL_TEST","pos":"00000000000000000000147","primary_keys":["id"],"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875,"address":"Shanghai"},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} {"table":"OGG.TBL_TEST","pos":"00000000000000000000152","primary_keys":["id"],"before":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875,"address":"Shanghai"},"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875,"address":"Beijing"},"op_type":"U","op_ts":"2020-05-13 17:26:27.936000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","primary_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","primary_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt index 5b6ef10854db..adeabb850ffb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-1.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt index da039452384e..84b9a8b27c04 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/ogg/table/startupmode/ogg-data-2.txt @@ -16,5 +16,5 @@ * limitations under the License. */ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} {"table":"OGG.TBL_TEST","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} \ No newline at end of file