[cdc-connector][sqlserver][tests] Fix UT errors by correcting right output (#2864)

pull/2893/head
Hongshun Wang 1 year ago committed by GitHub
parent 1839fb556c
commit 2c557c6a78
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

@ -398,7 +398,7 @@ public class MongoDBFullChangelogITCase extends MongoDBSourceTestBase {
env.enableCheckpointing(1000); env.enableCheckpointing(1000);
env.setParallelism(1); env.setParallelism(1);
ResolvedSchema customersSchame = ResolvedSchema customersSchema =
new ResolvedSchema( new ResolvedSchema(
Arrays.asList( Arrays.asList(
physical("cid", BIGINT().notNull()), physical("cid", BIGINT().notNull()),
@ -407,7 +407,7 @@ public class MongoDBFullChangelogITCase extends MongoDBSourceTestBase {
physical("phone_number", STRING())), physical("phone_number", STRING())),
new ArrayList<>(), new ArrayList<>(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("cid"))); UniqueConstraint.primaryKey("pk", Collections.singletonList("cid")));
TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchame); TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchema);
MongoDBSource source = MongoDBSource source =
new MongoDBSourceBuilder() new MongoDBSourceBuilder()
.hosts(CONTAINER.getHostAndPort()) .hosts(CONTAINER.getHostAndPort())

@ -302,7 +302,7 @@ public class MongoDBParallelSourceITCase extends MongoDBSourceTestBase {
env.enableCheckpointing(1000); env.enableCheckpointing(1000);
env.setParallelism(1); env.setParallelism(1);
ResolvedSchema customersSchame = ResolvedSchema customersSchema =
new ResolvedSchema( new ResolvedSchema(
Arrays.asList( Arrays.asList(
physical("cid", BIGINT().notNull()), physical("cid", BIGINT().notNull()),
@ -311,7 +311,7 @@ public class MongoDBParallelSourceITCase extends MongoDBSourceTestBase {
physical("phone_number", STRING())), physical("phone_number", STRING())),
new ArrayList<>(), new ArrayList<>(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("cid"))); UniqueConstraint.primaryKey("pk", Collections.singletonList("cid")));
TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchame); TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchema);
MongoDBSource source = MongoDBSource source =
new MongoDBSourceBuilder() new MongoDBSourceBuilder()
.hosts(CONTAINER.getHostAndPort()) .hosts(CONTAINER.getHostAndPort())
@ -345,11 +345,6 @@ public class MongoDBParallelSourceITCase extends MongoDBSourceTestBase {
mongoCollection.updateOne( mongoCollection.updateOne(
Filters.eq("cid", 2000L), Updates.set("address", "Pittsburgh")); Filters.eq("cid", 2000L), Updates.set("address", "Pittsburgh"));
mongoCollection.deleteOne(Filters.eq("cid", 1019L)); mongoCollection.deleteOne(Filters.eq("cid", 1019L));
try {
Thread.sleep(500L);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}; };
if (hookType == USE_POST_LOWWATERMARK_HOOK) { if (hookType == USE_POST_LOWWATERMARK_HOOK) {

@ -528,11 +528,6 @@ public class MySqlSourceITCase extends MySqlSourceTestBase {
connection.setAutoCommit(false); connection.setAutoCommit(false);
connection.execute(statements); connection.execute(statements);
connection.commit(); connection.commit();
try {
Thread.sleep(500L);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}; };
if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { if (hookType == USE_PRE_HIGHWATERMARK_HOOK) {
hooks.setPreHighWatermarkAction(snapshotPhaseHook); hooks.setPreHighWatermarkAction(snapshotPhaseHook);

@ -275,7 +275,7 @@ public class OracleSourceITCase extends OracleSourceTestBase {
env.enableCheckpointing(200L); env.enableCheckpointing(200L);
env.setParallelism(1); env.setParallelism(1);
ResolvedSchema customersSchame = ResolvedSchema customersSchema =
new ResolvedSchema( new ResolvedSchema(
Arrays.asList( Arrays.asList(
physical("ID", BIGINT().notNull()), physical("ID", BIGINT().notNull()),
@ -285,7 +285,7 @@ public class OracleSourceITCase extends OracleSourceTestBase {
new ArrayList<>(), new ArrayList<>(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("ID"))); UniqueConstraint.primaryKey("pk", Collections.singletonList("ID")));
TestTable customerTable = TestTable customerTable =
new TestTable(ORACLE_DATABASE, ORACLE_SCHEMA, "CUSTOMERS", customersSchame); new TestTable(ORACLE_DATABASE, ORACLE_SCHEMA, "CUSTOMERS", customersSchema);
String tableId = customerTable.getTableId(); String tableId = customerTable.getTableId();
OracleSourceBuilder.OracleIncrementalSource source = OracleSourceBuilder.OracleIncrementalSource source =
@ -326,9 +326,6 @@ public class OracleSourceITCase extends OracleSourceTestBase {
try (OracleConnection oracleConnection = try (OracleConnection oracleConnection =
OracleConnectionUtils.createOracleConnection(configuration)) { OracleConnectionUtils.createOracleConnection(configuration)) {
oracleConnection.execute(statements); oracleConnection.execute(statements);
Thread.sleep(500L);
} catch (InterruptedException e) {
throw new RuntimeException(e);
} }
}; };

@ -482,7 +482,7 @@ public class PostgresSourceITCase extends PostgresTestBase {
env.enableCheckpointing(1000); env.enableCheckpointing(1000);
env.setParallelism(1); env.setParallelism(1);
ResolvedSchema customersSchame = ResolvedSchema customersSchema =
new ResolvedSchema( new ResolvedSchema(
Arrays.asList( Arrays.asList(
physical("id", BIGINT().notNull()), physical("id", BIGINT().notNull()),
@ -492,7 +492,7 @@ public class PostgresSourceITCase extends PostgresTestBase {
new ArrayList<>(), new ArrayList<>(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("id"))); UniqueConstraint.primaryKey("pk", Collections.singletonList("id")));
TestTable customerTable = TestTable customerTable =
new TestTable(customDatabase, "customer", "customers", customersSchame); new TestTable(customDatabase, "customer", "customers", customersSchema);
String tableId = customerTable.getTableId(); String tableId = customerTable.getTableId();
PostgresSourceBuilder.PostgresIncrementalSource source = PostgresSourceBuilder.PostgresIncrementalSource source =
@ -525,9 +525,6 @@ public class PostgresSourceITCase extends PostgresTestBase {
try (PostgresConnection postgresConnection = dialect.openJdbcConnection()) { try (PostgresConnection postgresConnection = dialect.openJdbcConnection()) {
postgresConnection.execute(statements); postgresConnection.execute(statements);
postgresConnection.commit(); postgresConnection.commit();
Thread.sleep(500L);
} catch (InterruptedException e) {
throw new RuntimeException(e);
} }
}; };

@ -123,7 +123,7 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
@Test @Test
public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception {
List<String> records = testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK); List<String> records = testBackfillWhenWritingEvents(false, 25, USE_PRE_HIGHWATERMARK_HOOK);
List<String> expectedRecords = List<String> expectedRecords =
Arrays.asList( Arrays.asList(
@ -146,17 +146,23 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
"+I[1016, user_17, Shanghai, 123567891234]", "+I[1016, user_17, Shanghai, 123567891234]",
"+I[1017, user_18, Shanghai, 123567891234]", "+I[1017, user_18, Shanghai, 123567891234]",
"+I[1018, user_19, Shanghai, 123567891234]", "+I[1018, user_19, Shanghai, 123567891234]",
"+I[2000, user_21, Pittsburgh, 123567891234]", "+I[1019, user_20, Shanghai, 123567891234]",
"+I[15213, user_15213, Shanghai, 123567891234]"); "+I[2000, user_21, Shanghai, 123567891234]",
// when enable backfill, the wal log between [snapshot, high_watermark) will be "+I[15213, user_15213, Shanghai, 123567891234]",
// applied as snapshot image "-U[2000, user_21, Shanghai, 123567891234]",
"+U[2000, user_21, Pittsburgh, 123567891234]",
"-D[1019, user_20, Shanghai, 123567891234]");
// In sqlserver database, because the capture process extracts change data from the
// transaction log, there is a built-in latency between the time that a change is committed
// to a source table and the time that the change appears within its associated change
// table.Then in streaming phase, the log which should be ignored will be read again.
assertEqualsInAnyOrder(expectedRecords, records); assertEqualsInAnyOrder(expectedRecords, records);
} }
@Test @Test
public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception {
List<String> records = testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK); List<String> records = testBackfillWhenWritingEvents(false, 25, USE_POST_LOWWATERMARK_HOOK);
List<String> expectedRecords = List<String> expectedRecords =
Arrays.asList( Arrays.asList(
@ -180,9 +186,15 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
"+I[1017, user_18, Shanghai, 123567891234]", "+I[1017, user_18, Shanghai, 123567891234]",
"+I[1018, user_19, Shanghai, 123567891234]", "+I[1018, user_19, Shanghai, 123567891234]",
"+I[2000, user_21, Pittsburgh, 123567891234]", "+I[2000, user_21, Pittsburgh, 123567891234]",
"+I[15213, user_15213, Shanghai, 123567891234]"); "+I[15213, user_15213, Shanghai, 123567891234]",
// when enable backfill, the wal log between [low_watermark, snapshot) will be applied "+I[15213, user_15213, Shanghai, 123567891234]",
// as snapshot image "-U[2000, user_21, Shanghai, 123567891234]",
"+U[2000, user_21, Pittsburgh, 123567891234]",
"-D[1019, user_20, Shanghai, 123567891234]");
// In sqlserver database, because the capture process extracts change data from the
// transaction log, there is a built-in latency between the time that a change is committed
// to a source table and the time that the change appears within its associated change
// table.Then in streaming phase, the log which should be ignored will be read again.
assertEqualsInAnyOrder(expectedRecords, records); assertEqualsInAnyOrder(expectedRecords, records);
} }
@ -272,7 +284,7 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
env.enableCheckpointing(1000); env.enableCheckpointing(1000);
env.setParallelism(1); env.setParallelism(1);
ResolvedSchema customersSchame = ResolvedSchema customersSchema =
new ResolvedSchema( new ResolvedSchema(
Arrays.asList( Arrays.asList(
physical("id", BIGINT().notNull()), physical("id", BIGINT().notNull()),
@ -281,7 +293,7 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
physical("phone_number", STRING())), physical("phone_number", STRING())),
new ArrayList<>(), new ArrayList<>(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("id"))); UniqueConstraint.primaryKey("pk", Collections.singletonList("id")));
TestTable customerTable = new TestTable(databaseName, "dbo", "customers", customersSchame); TestTable customerTable = new TestTable(databaseName, "dbo", "customers", customersSchema);
String tableId = customerTable.getTableId(); String tableId = customerTable.getTableId();
SqlServerSourceBuilder.SqlServerIncrementalSource source = SqlServerSourceBuilder.SqlServerIncrementalSource source =
@ -310,14 +322,10 @@ public class SqlServerSourceITCase extends SqlServerSourceTestBase {
(sourceConfig, split) -> { (sourceConfig, split) -> {
SqlServerDialect dialect = SqlServerDialect dialect =
new SqlServerDialect((SqlServerSourceConfig) sourceConfig); new SqlServerDialect((SqlServerSourceConfig) sourceConfig);
JdbcConnection sqlServerConnection = try (JdbcConnection sqlServerConnection =
dialect.openJdbcConnection((JdbcSourceConfig) sourceConfig); dialect.openJdbcConnection((JdbcSourceConfig) sourceConfig)) {
sqlServerConnection.execute(statements); sqlServerConnection.execute(statements);
sqlServerConnection.commit(); sqlServerConnection.commit();
try {
Thread.sleep(5000L);
} catch (InterruptedException e) {
throw new RuntimeException(e);
} }
}; };

Loading…
Cancel
Save