|
|
|
@ -402,6 +402,78 @@ public class MySqlConnectorITCase extends MySqlParallelSourceTestBase {
|
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testWideTable() throws Exception {
|
|
|
|
|
final int tableColumnCount = 500;
|
|
|
|
|
fullTypesDatabase.createAndInitialize();
|
|
|
|
|
try (Connection connection = fullTypesDatabase.getJdbcConnection();
|
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
|
|
statement.execute(String.format("USE %s", fullTypesDatabase.getDatabaseName()));
|
|
|
|
|
statement.execute(
|
|
|
|
|
"CREATE TABLE wide_table("
|
|
|
|
|
+ buildColumnsDDL("col", 0, tableColumnCount, "BIGINT")
|
|
|
|
|
+ " PRIMARY KEY (col0) "
|
|
|
|
|
+ ")");
|
|
|
|
|
statement.execute(
|
|
|
|
|
"INSERT INTO wide_table values("
|
|
|
|
|
+ getIntegerSeqString(0, tableColumnCount)
|
|
|
|
|
+ ")");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
String sourceDDL =
|
|
|
|
|
String.format(
|
|
|
|
|
"CREATE TABLE wide_table (\n"
|
|
|
|
|
+ buildColumnsDDL("col", 0, tableColumnCount, "BIGINT")
|
|
|
|
|
+ " primary key (`col0`) not enforced"
|
|
|
|
|
+ ") WITH ("
|
|
|
|
|
+ " 'connector' = 'mysql-cdc',"
|
|
|
|
|
+ " 'hostname' = '%s',"
|
|
|
|
|
+ " 'port' = '%s',"
|
|
|
|
|
+ " 'username' = '%s',"
|
|
|
|
|
+ " 'password' = '%s',"
|
|
|
|
|
+ " 'database-name' = '%s',"
|
|
|
|
|
+ " 'table-name' = '%s',"
|
|
|
|
|
+ " 'debezium.internal.implementation' = '%s',"
|
|
|
|
|
+ " 'scan.incremental.snapshot.enabled' = '%s',"
|
|
|
|
|
+ " 'server-id' = '%s',"
|
|
|
|
|
+ " 'scan.incremental.snapshot.chunk.size' = '%s'"
|
|
|
|
|
+ ")",
|
|
|
|
|
MYSQL_CONTAINER.getHost(),
|
|
|
|
|
MYSQL_CONTAINER.getDatabasePort(),
|
|
|
|
|
fullTypesDatabase.getUsername(),
|
|
|
|
|
fullTypesDatabase.getPassword(),
|
|
|
|
|
fullTypesDatabase.getDatabaseName(),
|
|
|
|
|
"wide_table",
|
|
|
|
|
getDezImplementation(),
|
|
|
|
|
incrementalSnapshot,
|
|
|
|
|
getServerId(),
|
|
|
|
|
getSplitSize());
|
|
|
|
|
tEnv.executeSql(sourceDDL);
|
|
|
|
|
|
|
|
|
|
// async submit job
|
|
|
|
|
TableResult result = tEnv.executeSql("SELECT * FROM wide_table");
|
|
|
|
|
|
|
|
|
|
CloseableIterator<Row> iterator = result.collect();
|
|
|
|
|
waitForSnapshotStarted(iterator);
|
|
|
|
|
|
|
|
|
|
try (Connection connection = fullTypesDatabase.getJdbcConnection();
|
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
|
|
|
|
|
|
|
statement.execute("UPDATE wide_table SET col1 = 1024 WHERE col0=0;");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
new String[] {
|
|
|
|
|
"+I[0, 1, " + getIntegerSeqString(2, tableColumnCount) + "]",
|
|
|
|
|
"-U[0, 1, " + getIntegerSeqString(2, tableColumnCount) + "]",
|
|
|
|
|
"+U[0, 1024, " + getIntegerSeqString(2, tableColumnCount) + "]"
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
assertEqualsInAnyOrder(
|
|
|
|
|
Arrays.asList(expected), fetchRows(result.collect(), expected.length));
|
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testStartupFromLatestOffset() throws Exception {
|
|
|
|
|
inventoryDatabase.createAndInitialize();
|
|
|
|
@ -817,6 +889,24 @@ public class MySqlConnectorITCase extends MySqlParallelSourceTestBase {
|
|
|
|
|
return 0;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private static String buildColumnsDDL(
|
|
|
|
|
String columnPrefix, int start, int end, String dataType) {
|
|
|
|
|
StringBuilder stringBuilder = new StringBuilder();
|
|
|
|
|
for (int i = start; i < end; i++) {
|
|
|
|
|
stringBuilder.append(columnPrefix).append(i).append(" ").append(dataType).append(",");
|
|
|
|
|
}
|
|
|
|
|
return stringBuilder.toString();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private static String getIntegerSeqString(int start, int end) {
|
|
|
|
|
StringBuilder stringBuilder = new StringBuilder();
|
|
|
|
|
for (int i = start; i < end - 1; i++) {
|
|
|
|
|
stringBuilder.append(i).append(", ");
|
|
|
|
|
}
|
|
|
|
|
stringBuilder.append(end - 1);
|
|
|
|
|
return stringBuilder.toString();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
|
|
|
|
|
while (sinkSize(sinkName) == 0) {
|
|
|
|
|
Thread.sleep(100);
|
|
|
|
|