|
|
|
@ -107,6 +107,11 @@ public class BinlogSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
private final UniqueDatabase inventoryDatabase8 =
|
|
|
|
|
new UniqueDatabase(MYSQL8_CONTAINER, "inventory", TEST_USER, TEST_PASSWORD);
|
|
|
|
|
|
|
|
|
|
private static final MySqlContainer MYSQL_CONTAINER_NOGTID =
|
|
|
|
|
createMySqlContainer(MySqlVersion.V5_7, "docker/server/my.cnf");
|
|
|
|
|
private final UniqueDatabase customerDatabaseNoGtid =
|
|
|
|
|
new UniqueDatabase(MYSQL_CONTAINER_NOGTID, "customer", TEST_USER, TEST_PASSWORD);
|
|
|
|
|
|
|
|
|
|
private BinaryLogClient binaryLogClient;
|
|
|
|
|
private MySqlConnection mySqlConnection;
|
|
|
|
|
|
|
|
|
@ -891,6 +896,69 @@ public class BinlogSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
assertThat(eventFilter.test(event)).isFalse();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testBinlogOffsetCompareWithSnapshotAndBinlogPhase() throws Exception {
|
|
|
|
|
Startables.deepStart(Stream.of(MYSQL_CONTAINER_NOGTID)).join();
|
|
|
|
|
// Preparations
|
|
|
|
|
customerDatabaseNoGtid.createAndInitialize();
|
|
|
|
|
MySqlSourceConfig sourceConfig =
|
|
|
|
|
getConfig(
|
|
|
|
|
MYSQL_CONTAINER_NOGTID, customerDatabaseNoGtid, new String[] {"customers"});
|
|
|
|
|
binaryLogClient = DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration());
|
|
|
|
|
mySqlConnection = DebeziumUtils.createMySqlConnection(sourceConfig);
|
|
|
|
|
|
|
|
|
|
// step-1: split snapshot
|
|
|
|
|
List<MySqlSnapshotSplit> snapshotSplits =
|
|
|
|
|
getMySqlSplits(new String[] {"customers"}, sourceConfig, customerDatabaseNoGtid);
|
|
|
|
|
|
|
|
|
|
final StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
final SnapshotSplitReader snapshotSplitReader =
|
|
|
|
|
new SnapshotSplitReader(statefulTaskContext, 0);
|
|
|
|
|
|
|
|
|
|
// step-1: read snapshot splits firstly
|
|
|
|
|
List<SourceRecord> snapshotRecords = new ArrayList<>();
|
|
|
|
|
for (int i = 0; i < snapshotSplits.size(); i++) {
|
|
|
|
|
MySqlSplit sqlSplit = snapshotSplits.get(i);
|
|
|
|
|
List<SourceRecord> sourceRecords = pollRecordsFromReader(snapshotSplitReader, sqlSplit);
|
|
|
|
|
snapshotRecords.addAll(sourceRecords);
|
|
|
|
|
// mock binlog event after read chunk1
|
|
|
|
|
if (i == 0) {
|
|
|
|
|
mySqlConnection.execute(
|
|
|
|
|
"INSERT INTO "
|
|
|
|
|
+ customerDatabaseNoGtid.qualifiedTableName("customers")
|
|
|
|
|
+ " VALUES(999999, 'user_22','Shanghai','123567891234')");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
snapshotSplitReader.close();
|
|
|
|
|
|
|
|
|
|
// step-2: create binlog split reader
|
|
|
|
|
List<FinishedSnapshotSplitInfo> finishedSplitsInfo =
|
|
|
|
|
getFinishedSplitsInfo(snapshotSplits, snapshotRecords);
|
|
|
|
|
BinlogOffset startingOffset = getStartingOffsetOfBinlogSplit(finishedSplitsInfo);
|
|
|
|
|
Map<TableId, TableChange> tableSchemas = new HashMap<>();
|
|
|
|
|
for (MySqlSplit mySqlSplit : snapshotSplits) {
|
|
|
|
|
tableSchemas.putAll(mySqlSplit.getTableSchemas());
|
|
|
|
|
}
|
|
|
|
|
MySqlSplit binlogSplit =
|
|
|
|
|
new MySqlBinlogSplit(
|
|
|
|
|
"binlog-split",
|
|
|
|
|
startingOffset,
|
|
|
|
|
BinlogOffset.ofNonStopping(),
|
|
|
|
|
finishedSplitsInfo,
|
|
|
|
|
tableSchemas,
|
|
|
|
|
finishedSplitsInfo.size());
|
|
|
|
|
|
|
|
|
|
// step-3: test read binlog split
|
|
|
|
|
BinlogSplitReader binlogReader = new BinlogSplitReader(statefulTaskContext, 0);
|
|
|
|
|
binlogReader.submitSplit(binlogSplit);
|
|
|
|
|
|
|
|
|
|
List<SourceRecord> sourceRecords =
|
|
|
|
|
pollRecordsFromReader(binlogReader, RecordUtils::isDataChangeRecord);
|
|
|
|
|
MYSQL_CONTAINER_NOGTID.stop();
|
|
|
|
|
assertTrue(sourceRecords.isEmpty());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private BinlogSplitReader createBinlogReader(MySqlSourceConfig sourceConfig) {
|
|
|
|
|
return createBinlogReader(sourceConfig, false);
|
|
|
|
|
}
|
|
|
|
@ -945,6 +1013,29 @@ public class BinlogSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
return records;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<SourceRecord> pollRecordsFromReader(
|
|
|
|
|
SnapshotSplitReader reader, MySqlSplit sqlSplit) {
|
|
|
|
|
List<SourceRecord> records = new ArrayList<>();
|
|
|
|
|
if (reader.isFinished()) {
|
|
|
|
|
reader.submitSplit(sqlSplit);
|
|
|
|
|
}
|
|
|
|
|
Iterator<SourceRecords> res;
|
|
|
|
|
try {
|
|
|
|
|
while ((res = reader.pollSplitRecords()) != null) {
|
|
|
|
|
while (res.hasNext()) {
|
|
|
|
|
Iterator<SourceRecord> iterator = res.next().iterator();
|
|
|
|
|
while (iterator.hasNext()) {
|
|
|
|
|
SourceRecord sourceRecord = iterator.next();
|
|
|
|
|
records.add(sourceRecord);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
} catch (InterruptedException e) {
|
|
|
|
|
throw new RuntimeException("Polling action was interrupted", e);
|
|
|
|
|
}
|
|
|
|
|
return records;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<String> readBinlogSplits(
|
|
|
|
|
DataType dataType, BinlogSplitReader reader, int expectedSize) {
|
|
|
|
|
List<String> actual = new ArrayList<>();
|
|
|
|
@ -1173,10 +1264,10 @@ public class BinlogSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<MySqlSnapshotSplit> getMySqlSplits(
|
|
|
|
|
String[] captureTables, MySqlSourceConfig sourceConfig) {
|
|
|
|
|
String[] captureTables, MySqlSourceConfig sourceConfig, UniqueDatabase database) {
|
|
|
|
|
List<String> captureTableIds =
|
|
|
|
|
Arrays.stream(captureTables)
|
|
|
|
|
.map(tableName -> customerDatabase.getDatabaseName() + "." + tableName)
|
|
|
|
|
.map(tableName -> database.getDatabaseName() + "." + tableName)
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
List<TableId> remainingTables =
|
|
|
|
|
captureTableIds.stream().map(TableId::parse).collect(Collectors.toList());
|
|
|
|
@ -1202,6 +1293,11 @@ public class BinlogSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
return mySqlSplits;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<MySqlSnapshotSplit> getMySqlSplits(
|
|
|
|
|
String[] captureTables, MySqlSourceConfig sourceConfig) {
|
|
|
|
|
return getMySqlSplits(captureTables, sourceConfig, customerDatabase);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private MySqlSourceConfig getConfig(StartupOptions startupOptions, String[] captureTables) {
|
|
|
|
|
return getConfig(MYSQL_CONTAINER, customerDatabase, startupOptions, captureTables);
|
|
|
|
|
}
|
|
|
|
|