Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Feature][CDC] Support read no primary key table #6098

Merged
merged 1 commit into from
Dec 28, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -62,27 +62,42 @@ public Collection<SnapshotSplit> generateSplits(TableId tableId) {
long start = System.currentTimeMillis();

Column splitColumn = getSplitColumn(jdbc, dialect, tableId);
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
} catch (SQLException e) {
throw new RuntimeException("Failed to split chunks for table " + tableId, e);
}

// convert chunks into splits
List<SnapshotSplit> splits = new ArrayList<>();
SeaTunnelRowType splitType = getSplitType(splitColumn);
for (int i = 0; i < chunks.size(); i++) {
ChunkRange chunk = chunks.get(i);
SnapshotSplit split =
createSnapshotSplit(
jdbc,
tableId,
i,
splitType,
chunk.getChunkStart(),
chunk.getChunkEnd());
splits.add(split);
if (splitColumn == null) {
if (sourceConfig.isExactlyOnce()) {
throw new UnsupportedOperationException(
String.format(
"Exactly once is enabled, but not found primary key or unique key for table %s",
tableId));
}
SnapshotSplit singleSplit = createSnapshotSplit(jdbc, tableId, 0, null, null, null);
splits.add(singleSplit);
log.warn(
"No evenly split column found for table {}, use single split {}",
tableId,
singleSplit);
} else {
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
} catch (SQLException e) {
throw new RuntimeException("Failed to split chunks for table " + tableId, e);
}

// convert chunks into splits
SeaTunnelRowType splitType = getSplitType(splitColumn);
for (int i = 0; i < chunks.size(); i++) {
ChunkRange chunk = chunks.get(i);
SnapshotSplit split =
createSnapshotSplit(
jdbc,
tableId,
i,
splitType,
chunk.getChunkStart(),
chunk.getChunkEnd());
splits.add(split);
}
}

long end = System.currentTimeMillis();
Expand Down Expand Up @@ -371,6 +386,8 @@ protected Column getSplitColumn(
}
}
}
} else {
log.warn("No primary key found for table {}", tableId);
}

List<ConstraintKey> uniqueKeys = dialect.getUniqueKeys(jdbc, tableId);
Expand All @@ -389,16 +406,15 @@ protected Column getSplitColumn(
}
}
}
} else {
log.warn("No unique key found for table {}", tableId);
}
if (splitColumn != null) {
return splitColumn;
}

throw new UnsupportedOperationException(
String.format(
"Incremental snapshot for tables requires primary key/unique key,"
+ " but table %s doesn't have primary key.",
tableId));
log.warn("No evenly split column found for table {}", tableId);
return null;
}

protected String splitId(TableId tableId, int chunkId) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ private void createDataEventsForTable(
snapshotSplit.getSplitEnd() == null,
snapshotSplit.getSplitStart(),
snapshotSplit.getSplitEnd(),
snapshotSplit.getSplitKeyType().getTotalFields(),
snapshotSplit.getSplitKeyType(),
connectorConfig.getSnapshotFetchSize());
ResultSet rs = selectStatement.executeQuery()) {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,13 +291,14 @@ public static PreparedStatement readTableSplitDataStatement(
boolean isLastSplit,
Object[] splitStart,
Object[] splitEnd,
int primaryKeyNum,
SeaTunnelRowType splitKeyType,
int fetchSize) {
try {
final PreparedStatement statement = initStatement(jdbc, sql, fetchSize);
if (isFirstSplit && isLastSplit) {
return statement;
}
int primaryKeyNum = splitKeyType.getTotalFields();
if (isFirstSplit) {
for (int i = 0; i < primaryKeyNum; i++) {
statement.setObject(i + 1, splitEnd[i]);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ private void createDataEventsForTable(
snapshotSplit.getSplitEnd() == null,
snapshotSplit.getSplitStart(),
snapshotSplit.getSplitEnd(),
snapshotSplit.getSplitKeyType().getTotalFields(),
snapshotSplit.getSplitKeyType(),
connectorConfig.getSnapshotFetchSize());
ResultSet rs = selectStatement.executeQuery()) {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,13 +293,14 @@ public static PreparedStatement readTableSplitDataStatement(
boolean isLastSplit,
Object[] splitStart,
Object[] splitEnd,
int primaryKeyNum,
SeaTunnelRowType splitKeyType,
int fetchSize) {
try {
final PreparedStatement statement = initStatement(jdbc, sql, fetchSize);
if (isFirstSplit && isLastSplit) {
return statement;
}
int primaryKeyNum = splitKeyType.getTotalFields();
if (isFirstSplit) {
for (int i = 0; i < primaryKeyNum; i++) {
statement.setObject(i + 1, splitEnd[i]);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,8 @@ public class MysqlCDCIT extends TestSuiteBase implements TestResource {

private static final String SOURCE_TABLE_1 = "mysql_cdc_e2e_source_table";
private static final String SOURCE_TABLE_2 = "mysql_cdc_e2e_source_table2";
private static final String SOURCE_TABLE_NO_PRIMARY_KEY =
"mysql_cdc_e2e_source_table_no_primary_key";
private static final String SINK_TABLE = "mysql_cdc_e2e_sink_table";

private static MySqlContainer createMySqlContainer(MySqlVersion version) {
Expand Down Expand Up @@ -211,6 +213,48 @@ public void testMysqlCdcCheckDataWithDisableExactlyonce(TestContainer container)
});
}

@TestTemplate
public void testMysqlCdcCheckDataWithNoPrimaryKey(TestContainer container) {
// Clear related content to ensure that multiple operations are not affected
clearTable(MYSQL_DATABASE, SINK_TABLE);

CompletableFuture.supplyAsync(
() -> {
try {
container.executeJob("/mysqlcdc_to_mysql_with_no_primary_key.conf");
} catch (Exception e) {
log.error("Commit task exception :" + e.getMessage());
throw new RuntimeException(e);
}
return null;
});
await().atMost(60000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
log.info(query(getSinkQuerySQL(MYSQL_DATABASE, SINK_TABLE)).toString());
Assertions.assertIterableEquals(
query(
getSourceQuerySQL(
MYSQL_DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)),
query(getSinkQuerySQL(MYSQL_DATABASE, SINK_TABLE)));
});

// insert update delete
executeSql("DELETE FROM " + MYSQL_DATABASE + "." + SOURCE_TABLE_NO_PRIMARY_KEY);
upsertDeleteSourceTable(MYSQL_DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY);

// stream stage
await().atMost(60000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
Assertions.assertIterableEquals(
query(
getSourceQuerySQL(
MYSQL_DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)),
query(getSinkQuerySQL(MYSQL_DATABASE, SINK_TABLE)));
});
}

@TestTemplate
@DisabledOnContainer(
value = {},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,54 @@ CREATE TABLE mysql_cdc_e2e_source_table2
DEFAULT CHARSET = utf8mb4
COLLATE = utf8mb4_0900_ai_ci;

CREATE TABLE mysql_cdc_e2e_source_table_no_primary_key
(
`id` int NOT NULL,
`f_binary` binary(64) DEFAULT NULL,
`f_blob` blob,
`f_long_varbinary` mediumblob,
`f_longblob` longblob,
`f_tinyblob` tinyblob,
`f_varbinary` varbinary(100) DEFAULT NULL,
`f_smallint` smallint DEFAULT NULL,
`f_smallint_unsigned` smallint unsigned DEFAULT NULL,
`f_mediumint` mediumint DEFAULT NULL,
`f_mediumint_unsigned` mediumint unsigned DEFAULT NULL,
`f_int` int DEFAULT NULL,
`f_int_unsigned` int unsigned DEFAULT NULL,
`f_integer` int DEFAULT NULL,
`f_integer_unsigned` int unsigned DEFAULT NULL,
`f_bigint` bigint DEFAULT NULL,
`f_bigint_unsigned` bigint unsigned DEFAULT NULL,
`f_numeric` decimal(10, 0) DEFAULT NULL,
`f_decimal` decimal(10, 0) DEFAULT NULL,
`f_float` float DEFAULT NULL,
`f_double` double DEFAULT NULL,
`f_double_precision` double DEFAULT NULL,
`f_longtext` longtext,
`f_mediumtext` mediumtext,
`f_text` text,
`f_tinytext` tinytext,
`f_varchar` varchar(100) DEFAULT NULL,
`f_date` date DEFAULT NULL,
`f_datetime` datetime DEFAULT NULL,
`f_timestamp` timestamp NULL DEFAULT NULL,
`f_bit1` bit(1) DEFAULT NULL,
`f_bit64` bit(64) DEFAULT NULL,
`f_char` char(1) DEFAULT NULL,
`f_enum` enum ('enum1','enum2','enum3') DEFAULT NULL,
`f_mediumblob` mediumblob,
`f_long_varchar` mediumtext,
`f_real` double DEFAULT NULL,
`f_time` time DEFAULT NULL,
`f_tinyint` tinyint DEFAULT NULL,
`f_tinyint_unsigned` tinyint unsigned DEFAULT NULL,
`f_json` json DEFAULT NULL,
`f_year` year DEFAULT NULL
) ENGINE = InnoDB
DEFAULT CHARSET = utf8mb4
COLLATE = utf8mb4_0900_ai_ci;

CREATE TABLE mysql_cdc_e2e_sink_table
(
`id` int NOT NULL AUTO_INCREMENT,
Expand Down Expand Up @@ -174,6 +222,7 @@ CREATE TABLE mysql_cdc_e2e_sink_table

truncate table mysql_cdc_e2e_source_table;
truncate table mysql_cdc_e2e_source_table2;
truncate table mysql_cdc_e2e_source_table_no_primary_key;
truncate table mysql_cdc_e2e_sink_table;

INSERT INTO mysql_cdc_e2e_source_table ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,
Expand Down Expand Up @@ -238,6 +287,36 @@ VALUES ( 1, 0x616263740000000000000000000000000000000000000000000000000000000000
0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', 112.345,
'14:30:00', -128, 22, '{ "key": "value" }', 2021 );

INSERT INTO mysql_cdc_e2e_source_table_no_primary_key ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,
f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,
f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,
f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,
f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,
f_tinyint, f_tinyint_unsigned, f_json, f_year )
VALUES ( 1, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,
0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,
0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,
123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',
'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00',
'2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',
0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',
12.345, '14:30:00', -128, 255, '{ "key": "value" }', 2022 ),
( 2, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,
0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,
0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321,
123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',
'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',
1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',
0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',
112.345, '14:30:00', -128, 22, '{ "key": "value" }', 2013 ),
( 3, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,
0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,
0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, 123,
789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',
'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',
1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',
0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', 112.345,
'14:30:00', -128, 22, '{ "key": "value" }', 2021 );

CREATE DATABASE IF NOT EXISTS `mysql_cdc2`;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
#
# 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.
#
######
###### This config file is a demonstration of streaming processing in seatunnel config
######

env {
# You can set engine configuration here
parallelism = 1
job.mode = "STREAMING"
checkpoint.interval = 5000
}

source {
MySQL-CDC {
result_table_name = "customers_mysql_cdc"
server-id = 5652
username = "st_user"
password = "seatunnel"
table-names = ["mysql_cdc.mysql_cdc_e2e_source_table_no_primary_key"]
base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc"

exactly_once = false
}
}

sink {
jdbc {
source_table_name = "customers_mysql_cdc"
url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc"
driver = "com.mysql.cj.jdbc.Driver"
user = "st_user"
password = "seatunnel"

generate_sink_sql = true
# You need to configure both database and table
database = mysql_cdc
table = mysql_cdc_e2e_sink_table
primary_keys = ["id"]
}
}
Loading