|
|
@ -24,31 +24,26 @@ import org.apache.flink.table.api.EnvironmentSettings;
|
|
|
|
import org.apache.flink.table.api.TableResult;
|
|
|
|
import org.apache.flink.table.api.TableResult;
|
|
|
|
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
|
|
|
|
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
|
|
|
|
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
|
|
|
|
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
|
|
|
|
import org.apache.flink.table.utils.LegacyRowResource;
|
|
|
|
|
|
|
|
import org.apache.flink.types.Row;
|
|
|
|
import org.apache.flink.types.Row;
|
|
|
|
|
|
|
|
import org.apache.flink.util.CloseableIterator;
|
|
|
|
|
|
|
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.MySqlTestBase;
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.MySqlTestBase;
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.utils.UniqueDatabase;
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.utils.UniqueDatabase;
|
|
|
|
import org.junit.Before;
|
|
|
|
import org.junit.Before;
|
|
|
|
import org.junit.ClassRule;
|
|
|
|
|
|
|
|
import org.junit.Test;
|
|
|
|
import org.junit.Test;
|
|
|
|
import org.junit.runner.RunWith;
|
|
|
|
import org.junit.runner.RunWith;
|
|
|
|
import org.junit.runners.Parameterized;
|
|
|
|
import org.junit.runners.Parameterized;
|
|
|
|
|
|
|
|
|
|
|
|
import java.sql.Connection;
|
|
|
|
import java.sql.Connection;
|
|
|
|
import java.sql.SQLException;
|
|
|
|
|
|
|
|
import java.sql.Statement;
|
|
|
|
import java.sql.Statement;
|
|
|
|
import java.util.ArrayList;
|
|
|
|
import java.util.ArrayList;
|
|
|
|
import java.util.Arrays;
|
|
|
|
|
|
|
|
import java.util.Iterator;
|
|
|
|
import java.util.Iterator;
|
|
|
|
import java.util.List;
|
|
|
|
import java.util.List;
|
|
|
|
import java.util.Random;
|
|
|
|
import java.util.Random;
|
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.MySqlSourceTest.currentMySQLLatestOffset;
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.MySqlSourceTest.currentMySQLLatestOffset;
|
|
|
|
import static org.apache.flink.api.common.JobStatus.RUNNING;
|
|
|
|
import static org.apache.flink.api.common.JobStatus.RUNNING;
|
|
|
|
import static org.hamcrest.Matchers.containsInAnyOrder;
|
|
|
|
import static org.hamcrest.Matchers.containsInAnyOrder;
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
|
|
import static org.junit.Assert.assertThat;
|
|
|
|
import static org.junit.Assert.assertThat;
|
|
|
|
|
|
|
|
|
|
|
|
/** Integration tests for MySQL binlog SQL source. */
|
|
|
|
/** Integration tests for MySQL binlog SQL source. */
|
|
|
@ -67,14 +62,13 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
StreamTableEnvironment.create(
|
|
|
|
StreamTableEnvironment.create(
|
|
|
|
env,
|
|
|
|
env,
|
|
|
|
EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build());
|
|
|
|
EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build());
|
|
|
|
|
|
|
|
|
|
|
|
// the debezium mysql connector use legacy implementation or not
|
|
|
|
// the debezium mysql connector use legacy implementation or not
|
|
|
|
private final boolean useLegacyDezMySQL;
|
|
|
|
private final boolean useLegacyDezMySQL;
|
|
|
|
|
|
|
|
|
|
|
|
// enable the parallelRead(i.e: The new source MySQLParallelSource)
|
|
|
|
// enable the parallelRead(i.e: The new source MySQLParallelSource)
|
|
|
|
private final boolean parallelRead;
|
|
|
|
private final boolean parallelRead;
|
|
|
|
|
|
|
|
|
|
|
|
@ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE;
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
public MySqlConnectorITCase(boolean useLegacyDezMySQL, boolean parallelRead) {
|
|
|
|
public MySqlConnectorITCase(boolean useLegacyDezMySQL, boolean parallelRead) {
|
|
|
|
this.useLegacyDezMySQL = useLegacyDezMySQL;
|
|
|
|
this.useLegacyDezMySQL = useLegacyDezMySQL;
|
|
|
|
this.parallelRead = parallelRead;
|
|
|
|
this.parallelRead = parallelRead;
|
|
|
@ -102,17 +96,16 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
@Test
|
|
|
|
public void testConsumingAllEvents()
|
|
|
|
public void testConsumingAllEvents() throws Exception {
|
|
|
|
throws SQLException, ExecutionException, InterruptedException {
|
|
|
|
|
|
|
|
inventoryDatabase.createAndInitialize();
|
|
|
|
inventoryDatabase.createAndInitialize();
|
|
|
|
String sourceDDL =
|
|
|
|
String sourceDDL =
|
|
|
|
String.format(
|
|
|
|
String.format(
|
|
|
|
"CREATE TABLE debezium_source ("
|
|
|
|
"CREATE TABLE debezium_source ("
|
|
|
|
+ " id INT NOT NULL,"
|
|
|
|
+ " `id` INT NOT NULL,"
|
|
|
|
+ " name STRING,"
|
|
|
|
+ " name STRING,"
|
|
|
|
+ " description STRING,"
|
|
|
|
+ " description STRING,"
|
|
|
|
+ " weight DECIMAL(10,3),"
|
|
|
|
+ " weight DECIMAL(10,3),"
|
|
|
|
+ " primary key (id) not enforced"
|
|
|
|
+ " primary key (`id`) not enforced"
|
|
|
|
+ ") WITH ("
|
|
|
|
+ ") WITH ("
|
|
|
|
+ " 'connector' = 'mysql-cdc',"
|
|
|
|
+ " 'connector' = 'mysql-cdc',"
|
|
|
|
+ " 'hostname' = '%s',"
|
|
|
|
+ " 'hostname' = '%s',"
|
|
|
@ -198,18 +191,17 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
String[] expected =
|
|
|
|
new String[] {
|
|
|
|
new String[] {
|
|
|
|
"scooter,3.140",
|
|
|
|
"+I[scooter, 3.140]",
|
|
|
|
"car battery,8.100",
|
|
|
|
"+I[car battery, 8.100]",
|
|
|
|
"12-pack drill bits,0.800",
|
|
|
|
"+I[12-pack drill bits, 0.800]",
|
|
|
|
"hammer,2.625",
|
|
|
|
"+I[hammer, 2.625]",
|
|
|
|
"rocks,5.100",
|
|
|
|
"+I[rocks, 5.100]",
|
|
|
|
"jacket,0.600",
|
|
|
|
"+I[jacket, 0.600]",
|
|
|
|
"spare tire,22.200"
|
|
|
|
"+I[spare tire, 22.200]"
|
|
|
|
};
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
|
|
|
|
|
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
@ -265,41 +257,12 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
parallelRead,
|
|
|
|
parallelRead,
|
|
|
|
getServerId(),
|
|
|
|
getServerId(),
|
|
|
|
getSplitSize());
|
|
|
|
getSplitSize());
|
|
|
|
String sinkDDL =
|
|
|
|
|
|
|
|
"CREATE TABLE sink (\n"
|
|
|
|
|
|
|
|
+ " id INT NOT NULL,\n"
|
|
|
|
|
|
|
|
+ " tiny_c TINYINT,\n"
|
|
|
|
|
|
|
|
+ " tiny_un_c SMALLINT ,\n"
|
|
|
|
|
|
|
|
+ " small_c SMALLINT,\n"
|
|
|
|
|
|
|
|
+ " small_un_c INT,\n"
|
|
|
|
|
|
|
|
+ " int_c INT ,\n"
|
|
|
|
|
|
|
|
+ " int_un_c BIGINT,\n"
|
|
|
|
|
|
|
|
+ " int11_c BIGINT,\n"
|
|
|
|
|
|
|
|
+ " big_c BIGINT,\n"
|
|
|
|
|
|
|
|
+ " varchar_c STRING,\n"
|
|
|
|
|
|
|
|
+ " char_c STRING,\n"
|
|
|
|
|
|
|
|
+ " float_c FLOAT,\n"
|
|
|
|
|
|
|
|
+ " double_c DOUBLE,\n"
|
|
|
|
|
|
|
|
+ " decimal_c DECIMAL(8, 4),\n"
|
|
|
|
|
|
|
|
+ " numeric_c DECIMAL(6, 0),\n"
|
|
|
|
|
|
|
|
+ " boolean_c BOOLEAN,\n"
|
|
|
|
|
|
|
|
+ " date_c DATE,\n"
|
|
|
|
|
|
|
|
+ " time_c TIME(0),\n"
|
|
|
|
|
|
|
|
+ " datetime3_c TIMESTAMP(3),\n"
|
|
|
|
|
|
|
|
+ " datetime6_c TIMESTAMP(6),\n"
|
|
|
|
|
|
|
|
+ " timestamp_c TIMESTAMP(0),\n"
|
|
|
|
|
|
|
|
+ " file_uuid STRING\n"
|
|
|
|
|
|
|
|
+ ") WITH ("
|
|
|
|
|
|
|
|
+ " 'connector' = 'values',"
|
|
|
|
|
|
|
|
+ " 'sink-insert-only' = 'false'"
|
|
|
|
|
|
|
|
+ ")";
|
|
|
|
|
|
|
|
tEnv.executeSql(sourceDDL);
|
|
|
|
tEnv.executeSql(sourceDDL);
|
|
|
|
tEnv.executeSql(sinkDDL);
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// async submit job
|
|
|
|
// async submit job
|
|
|
|
TableResult result =
|
|
|
|
TableResult result =
|
|
|
|
tEnv.executeSql(
|
|
|
|
tEnv.executeSql(
|
|
|
|
"INSERT INTO sink SELECT id,\n"
|
|
|
|
"SELECT id,\n"
|
|
|
|
+ "tiny_c,\n"
|
|
|
|
+ "tiny_c,\n"
|
|
|
|
+ "tiny_un_c,\n"
|
|
|
|
+ "tiny_un_c,\n"
|
|
|
|
+ "small_c,\n"
|
|
|
|
+ "small_c,\n"
|
|
|
@ -322,7 +285,8 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
+ "timestamp_c,\n"
|
|
|
|
+ "timestamp_c,\n"
|
|
|
|
+ "TO_BASE64(DECODE(file_uuid, 'UTF-8')) FROM full_types");
|
|
|
|
+ "TO_BASE64(DECODE(file_uuid, 'UTF-8')) FROM full_types");
|
|
|
|
|
|
|
|
|
|
|
|
waitForSnapshotStarted("sink");
|
|
|
|
CloseableIterator<Row> iterator = result.collect();
|
|
|
|
|
|
|
|
waitForSnapshotStarted(iterator);
|
|
|
|
|
|
|
|
|
|
|
|
try (Connection connection = fullTypesDatabase.getJdbcConnection();
|
|
|
|
try (Connection connection = fullTypesDatabase.getJdbcConnection();
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
@ -331,22 +295,85 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
"UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
|
|
|
|
"UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
waitForSinkSize("sink", 3);
|
|
|
|
String[] expected =
|
|
|
|
|
|
|
|
new String[] {
|
|
|
|
List<String> expected =
|
|
|
|
"+I[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true, 2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:00:22, ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=]",
|
|
|
|
Arrays.asList(
|
|
|
|
"-U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true, 2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:00:22, ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=]",
|
|
|
|
"+I(1,127,255,32767,65535,2147483647,4294967295,2147483647,9223372036854775807,Hello World,abc,"
|
|
|
|
"+U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true, 2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:33:22, ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=]"
|
|
|
|
+ "123.102,404.4443,123.4567,346,true,2020-07-17,18:00:22,2020-07-17T18:00:22.123,"
|
|
|
|
};
|
|
|
|
+ "2020-07-17T18:00:22.123456,2020-07-17T18:00:22,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=)",
|
|
|
|
|
|
|
|
"-U(1,127,255,32767,65535,2147483647,4294967295,2147483647,9223372036854775807,Hello World,abc,"
|
|
|
|
|
|
|
|
+ "123.102,404.4443,123.4567,346,true,2020-07-17,18:00:22,2020-07-17T18:00:22.123,"
|
|
|
|
|
|
|
|
+ "2020-07-17T18:00:22.123456,2020-07-17T18:00:22,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=)",
|
|
|
|
|
|
|
|
"+U(1,127,255,32767,65535,2147483647,4294967295,2147483647,9223372036854775807,Hello World,abc,"
|
|
|
|
|
|
|
|
+ "123.102,404.4443,123.4567,346,true,2020-07-17,18:00:22,2020-07-17T18:00:22.123,"
|
|
|
|
|
|
|
|
+ "2020-07-17T18:00:22.123456,2020-07-17T18:33:22,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=)");
|
|
|
|
|
|
|
|
List<String> actual = TestValuesTableFactory.getRawResults("sink");
|
|
|
|
|
|
|
|
assertEquals(expected, actual);
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
assertThat(fetchRows(result.collect(), 3), containsInAnyOrder(expected));
|
|
|
|
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
|
|
|
public void testStartupFromLatestOffset() throws Exception {
|
|
|
|
|
|
|
|
inventoryDatabase.createAndInitialize();
|
|
|
|
|
|
|
|
String sourceDDL =
|
|
|
|
|
|
|
|
String.format(
|
|
|
|
|
|
|
|
"CREATE TABLE debezium_source ("
|
|
|
|
|
|
|
|
+ " id INT NOT NULL,"
|
|
|
|
|
|
|
|
+ " name STRING,"
|
|
|
|
|
|
|
|
+ " description STRING,"
|
|
|
|
|
|
|
|
+ " weight DECIMAL(10,3),"
|
|
|
|
|
|
|
|
+ " primary key(id) not enforced"
|
|
|
|
|
|
|
|
+ ") WITH ("
|
|
|
|
|
|
|
|
+ " 'connector' = 'mysql-cdc',"
|
|
|
|
|
|
|
|
+ " 'hostname' = '%s',"
|
|
|
|
|
|
|
|
+ " 'port' = '%s',"
|
|
|
|
|
|
|
|
+ " 'username' = '%s',"
|
|
|
|
|
|
|
|
+ " 'password' = '%s',"
|
|
|
|
|
|
|
|
+ " 'database-name' = '%s',"
|
|
|
|
|
|
|
|
+ " 'table-name' = '%s',"
|
|
|
|
|
|
|
|
+ " 'scan.startup.mode' = 'latest-offset',"
|
|
|
|
|
|
|
|
+ " 'scan.snapshot.parallel-read' = '%s',"
|
|
|
|
|
|
|
|
+ " 'server-id' = '%s',"
|
|
|
|
|
|
|
|
+ " 'debezium.internal.implementation' = '%s'"
|
|
|
|
|
|
|
|
+ ")",
|
|
|
|
|
|
|
|
MYSQL_CONTAINER.getHost(),
|
|
|
|
|
|
|
|
MYSQL_CONTAINER.getDatabasePort(),
|
|
|
|
|
|
|
|
inventoryDatabase.getUsername(),
|
|
|
|
|
|
|
|
inventoryDatabase.getPassword(),
|
|
|
|
|
|
|
|
inventoryDatabase.getDatabaseName(),
|
|
|
|
|
|
|
|
"products",
|
|
|
|
|
|
|
|
parallelRead,
|
|
|
|
|
|
|
|
getServerId(),
|
|
|
|
|
|
|
|
getDezImplementation());
|
|
|
|
|
|
|
|
tEnv.executeSql(sourceDDL);
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// async submit job
|
|
|
|
|
|
|
|
TableResult result = tEnv.executeSql("SELECT * FROM debezium_source");
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// wait for the source startup, we don't have a better way to wait it, use sleep for now
|
|
|
|
|
|
|
|
while (result.getJobClient().get().getJobStatus().get() != RUNNING) {
|
|
|
|
|
|
|
|
Thread.sleep(5000L);
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
CloseableIterator<Row> iterator = result.collect();
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
try (Connection connection = inventoryDatabase.getJdbcConnection();
|
|
|
|
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);");
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;");
|
|
|
|
|
|
|
|
statement.execute("UPDATE products SET weight='5.17' WHERE id=111;");
|
|
|
|
|
|
|
|
statement.execute("DELETE FROM products WHERE id=111;");
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
|
|
|
new String[] {
|
|
|
|
|
|
|
|
"+I[110, jacket, water resistent white wind breaker, 0.200]",
|
|
|
|
|
|
|
|
"+I[111, scooter, Big 2-wheel scooter , 5.180]",
|
|
|
|
|
|
|
|
"-U[110, jacket, water resistent white wind breaker, 0.200]",
|
|
|
|
|
|
|
|
"+U[110, jacket, new water resistent white wind breaker, 0.500]",
|
|
|
|
|
|
|
|
"-U[111, scooter, Big 2-wheel scooter , 5.180]",
|
|
|
|
|
|
|
|
"+U[111, scooter, Big 2-wheel scooter , 5.170]",
|
|
|
|
|
|
|
|
"-D[111, scooter, Big 2-wheel scooter , 5.170]"
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
assertThat(fetchRows(iterator, expected.length), containsInAnyOrder(expected));
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
@ -431,7 +458,7 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
waitForSinkSize("sink", 7);
|
|
|
|
waitForSinkSize("sink", 7);
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
String[] expected =
|
|
|
|
new String[] {"110,jacket,new water resistent white wind breaker,0.500"};
|
|
|
|
new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
|
|
|
|
|
|
|
|
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
@ -505,16 +532,16 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
String[] expected =
|
|
|
|
new String[] {
|
|
|
|
new String[] {
|
|
|
|
"101,scooter,Small 2-wheel scooter,3.140",
|
|
|
|
"+I[101, scooter, Small 2-wheel scooter, 3.140]",
|
|
|
|
"102,car battery,12V car battery,8.100",
|
|
|
|
"+I[102, car battery, 12V car battery, 8.100]",
|
|
|
|
"103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.800",
|
|
|
|
"+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800]",
|
|
|
|
"104,hammer,12oz carpenter's hammer,0.750",
|
|
|
|
"+I[104, hammer, 12oz carpenter's hammer, 0.750]",
|
|
|
|
"105,hammer,14oz carpenter's hammer,0.875",
|
|
|
|
"+I[105, hammer, 14oz carpenter's hammer, 0.875]",
|
|
|
|
"106,hammer,18oz carpenter hammer,1.000",
|
|
|
|
"+I[108, jacket, water resistent black wind breaker, 0.100]",
|
|
|
|
"107,rocks,box of assorted rocks,5.100",
|
|
|
|
"+I[109, spare tire, 24 inch spare tire, 22.200]",
|
|
|
|
"108,jacket,water resistent black wind breaker,0.100",
|
|
|
|
"+I[106, hammer, 18oz carpenter hammer, 1.000]",
|
|
|
|
"109,spare tire,24 inch spare tire,22.200",
|
|
|
|
"+I[107, rocks, box of assorted rocks, 5.100]",
|
|
|
|
"110,jacket,new water resistent white wind breaker,0.500"
|
|
|
|
"+I[110, jacket, new water resistent white wind breaker, 0.500]"
|
|
|
|
};
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
@ -523,85 +550,6 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
|
|
|
public void testStartupFromLatestOffset() throws Exception {
|
|
|
|
|
|
|
|
inventoryDatabase.createAndInitialize();
|
|
|
|
|
|
|
|
String sourceDDL =
|
|
|
|
|
|
|
|
String.format(
|
|
|
|
|
|
|
|
"CREATE TABLE debezium_source ("
|
|
|
|
|
|
|
|
+ " id INT NOT NULL,"
|
|
|
|
|
|
|
|
+ " name STRING,"
|
|
|
|
|
|
|
|
+ " description STRING,"
|
|
|
|
|
|
|
|
+ " weight DECIMAL(10,3),"
|
|
|
|
|
|
|
|
+ " primary key(id) not enforced"
|
|
|
|
|
|
|
|
+ ") WITH ("
|
|
|
|
|
|
|
|
+ " 'connector' = 'mysql-cdc',"
|
|
|
|
|
|
|
|
+ " 'hostname' = '%s',"
|
|
|
|
|
|
|
|
+ " 'port' = '%s',"
|
|
|
|
|
|
|
|
+ " 'username' = '%s',"
|
|
|
|
|
|
|
|
+ " 'password' = '%s',"
|
|
|
|
|
|
|
|
+ " 'database-name' = '%s',"
|
|
|
|
|
|
|
|
+ " 'table-name' = '%s',"
|
|
|
|
|
|
|
|
+ " 'scan.startup.mode' = 'latest-offset',"
|
|
|
|
|
|
|
|
+ " 'scan.snapshot.parallel-read' = '%s',"
|
|
|
|
|
|
|
|
+ " 'server-id' = '%s',"
|
|
|
|
|
|
|
|
+ " 'debezium.internal.implementation' = '%s'"
|
|
|
|
|
|
|
|
+ ")",
|
|
|
|
|
|
|
|
MYSQL_CONTAINER.getHost(),
|
|
|
|
|
|
|
|
MYSQL_CONTAINER.getDatabasePort(),
|
|
|
|
|
|
|
|
inventoryDatabase.getUsername(),
|
|
|
|
|
|
|
|
inventoryDatabase.getPassword(),
|
|
|
|
|
|
|
|
inventoryDatabase.getDatabaseName(),
|
|
|
|
|
|
|
|
"products",
|
|
|
|
|
|
|
|
parallelRead,
|
|
|
|
|
|
|
|
getServerId(),
|
|
|
|
|
|
|
|
getDezImplementation());
|
|
|
|
|
|
|
|
tEnv.executeSql(sourceDDL);
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// async submit job
|
|
|
|
|
|
|
|
TableResult result = tEnv.executeSql("SELECT * FROM debezium_source");
|
|
|
|
|
|
|
|
// wait for the source startup, we don't have a better way to wait it, use sleep for now
|
|
|
|
|
|
|
|
while (result.getJobClient().get().getJobStatus().get() != RUNNING) {
|
|
|
|
|
|
|
|
Thread.sleep(5000L);
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
try (Connection connection = inventoryDatabase.getJdbcConnection();
|
|
|
|
|
|
|
|
Statement statement = connection.createStatement()) {
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);");
|
|
|
|
|
|
|
|
statement.execute(
|
|
|
|
|
|
|
|
"UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;");
|
|
|
|
|
|
|
|
statement.execute("UPDATE products SET weight='5.17' WHERE id=111;");
|
|
|
|
|
|
|
|
statement.execute("DELETE FROM products WHERE id=111;");
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
|
|
|
new String[] {
|
|
|
|
|
|
|
|
"110,jacket,water resistent white wind breaker,0.200",
|
|
|
|
|
|
|
|
"111,scooter,Big 2-wheel scooter ,5.180",
|
|
|
|
|
|
|
|
"110,jacket,water resistent white wind breaker,0.200",
|
|
|
|
|
|
|
|
"110,jacket,new water resistent white wind breaker,0.500",
|
|
|
|
|
|
|
|
"111,scooter,Big 2-wheel scooter ,5.180",
|
|
|
|
|
|
|
|
"111,scooter,Big 2-wheel scooter ,5.170",
|
|
|
|
|
|
|
|
"111,scooter,Big 2-wheel scooter ,5.170"
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
assertThat(fetchRows(result.collect(), 7), containsInAnyOrder(expected));
|
|
|
|
|
|
|
|
result.getJobClient().get().cancel().get();
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private static List<String> fetchRows(Iterator<Row> iter, int size) {
|
|
|
|
|
|
|
|
List<String> rows = new ArrayList<>(size);
|
|
|
|
|
|
|
|
while (size > 0 && iter.hasNext()) {
|
|
|
|
|
|
|
|
Row row = iter.next();
|
|
|
|
|
|
|
|
rows.add(row.toString());
|
|
|
|
|
|
|
|
size--;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
return rows;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
@Test
|
|
|
|
public void testStartupFromTimestamp() throws Exception {
|
|
|
|
public void testStartupFromTimestamp() throws Exception {
|
|
|
|
if (parallelRead) {
|
|
|
|
if (parallelRead) {
|
|
|
@ -668,7 +616,7 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
waitForSinkSize("sink", 7);
|
|
|
|
waitForSinkSize("sink", 7);
|
|
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
String[] expected =
|
|
|
|
new String[] {"110,jacket,new water resistent white wind breaker,0.500"};
|
|
|
|
new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
|
|
|
|
|
|
|
|
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
|
assertThat(actual, containsInAnyOrder(expected));
|
|
|
@ -722,4 +670,20 @@ public class MySqlConnectorITCase extends MySqlTestBase {
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private static List<String> fetchRows(Iterator<Row> iter, int size) {
|
|
|
|
|
|
|
|
List<String> rows = new ArrayList<>(size);
|
|
|
|
|
|
|
|
while (size > 0 && iter.hasNext()) {
|
|
|
|
|
|
|
|
Row row = iter.next();
|
|
|
|
|
|
|
|
rows.add(row.toString());
|
|
|
|
|
|
|
|
size--;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
return rows;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private static void waitForSnapshotStarted(CloseableIterator<Row> iterator) throws Exception {
|
|
|
|
|
|
|
|
while (!iterator.hasNext()) {
|
|
|
|
|
|
|
|
Thread.sleep(100);
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|