|
|
|
@ -24,7 +24,7 @@ import org.apache.flink.table.types.logical.BigIntType;
|
|
|
|
|
import org.apache.flink.table.types.logical.RowType;
|
|
|
|
|
import org.apache.flink.table.types.logical.VarCharType;
|
|
|
|
|
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.debezium.offset.BinlogPosition;
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.split.MySQLSplit;
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.split.MySQLSplitKind;
|
|
|
|
|
import com.alibaba.ververica.cdc.connectors.mysql.source.split.MySQLSplitSerializer;
|
|
|
|
@ -39,7 +39,7 @@ import java.util.HashMap;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.Map;
|
|
|
|
|
|
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.debezium.offset.BinlogPosition.INITIAL_OFFSET;
|
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.source.offset.BinlogOffset.INITIAL_OFFSET;
|
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.source.split.MySQLSplitSerializerTest.assertSplitsEqual;
|
|
|
|
|
import static com.alibaba.ververica.cdc.connectors.mysql.source.split.MySQLSplitSerializerTest.getTestHistoryRecord;
|
|
|
|
|
import static org.junit.Assert.assertArrayEquals;
|
|
|
|
@ -103,13 +103,13 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
assignedSplits.put(0, assignedSplitsForTask0);
|
|
|
|
|
assignedSplits.put(1, assignedSplitsForTask1);
|
|
|
|
|
|
|
|
|
|
final Map<Integer, List<Tuple2<String, BinlogPosition>>> finishedSnapshotSplits =
|
|
|
|
|
final Map<Integer, List<Tuple2<String, BinlogOffset>>> finishedSnapshotSplits =
|
|
|
|
|
new HashMap<>();
|
|
|
|
|
List<Tuple2<String, BinlogPosition>> finishedSplitsForTask0 = new ArrayList<>();
|
|
|
|
|
List<Tuple2<String, BinlogOffset>> finishedSplitsForTask0 = new ArrayList<>();
|
|
|
|
|
finishedSplitsForTask0.add(getTestSplitInfo(tableId0, 0));
|
|
|
|
|
finishedSplitsForTask0.add(getTestSplitInfo(tableId0, 1));
|
|
|
|
|
finishedSplitsForTask0.add(getTestSplitInfo(tableId1, 0));
|
|
|
|
|
List<Tuple2<String, BinlogPosition>> finishedSplitsForTask1 = new ArrayList<>();
|
|
|
|
|
List<Tuple2<String, BinlogOffset>> finishedSplitsForTask1 = new ArrayList<>();
|
|
|
|
|
finishedSplitsForTask0.add(getTestSplitInfo(tableId1, 1));
|
|
|
|
|
finishedSplitsForTask0.add(getTestSplitInfo(tableId0, 2));
|
|
|
|
|
finishedSnapshotSplits.put(0, finishedSplitsForTask0);
|
|
|
|
@ -127,8 +127,8 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
new RowType(Arrays.asList(new RowType.RowField("id", new BigIntType()))),
|
|
|
|
|
new Object[] {100L + splitNo * 1000},
|
|
|
|
|
new Object[] {999L + splitNo * 1000},
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 3L + splitNo * 200),
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 78L + splitNo * 200),
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 3L + splitNo * 200),
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 78L + splitNo * 200),
|
|
|
|
|
true,
|
|
|
|
|
INITIAL_OFFSET,
|
|
|
|
|
new ArrayList<>(),
|
|
|
|
@ -136,7 +136,7 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private MySQLSplit getTestBinlogSplit(TableId tableId) throws Exception {
|
|
|
|
|
final List<Tuple5<TableId, String, Object[], Object[], BinlogPosition>> finishedSplitsInfo =
|
|
|
|
|
final List<Tuple5<TableId, String, Object[], Object[], BinlogOffset>> finishedSplitsInfo =
|
|
|
|
|
new ArrayList<>();
|
|
|
|
|
finishedSplitsInfo.add(
|
|
|
|
|
Tuple5.of(
|
|
|
|
@ -144,21 +144,21 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
tableId + "-0",
|
|
|
|
|
null,
|
|
|
|
|
new Object[] {100},
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 0L)));
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 0L)));
|
|
|
|
|
finishedSplitsInfo.add(
|
|
|
|
|
Tuple5.of(
|
|
|
|
|
tableId,
|
|
|
|
|
tableId + "-1",
|
|
|
|
|
new Object[] {100},
|
|
|
|
|
new Object[] {200},
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 200L)));
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 200L)));
|
|
|
|
|
finishedSplitsInfo.add(
|
|
|
|
|
Tuple5.of(
|
|
|
|
|
tableId,
|
|
|
|
|
tableId + "-2",
|
|
|
|
|
new Object[] {200},
|
|
|
|
|
null,
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 400L)));
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 400L)));
|
|
|
|
|
|
|
|
|
|
final Map<TableId, SchemaRecord> databaseHistory = new HashMap<>();
|
|
|
|
|
databaseHistory.put(tableId, getTestHistoryRecord());
|
|
|
|
@ -172,15 +172,14 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
null,
|
|
|
|
|
null,
|
|
|
|
|
true,
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 0L),
|
|
|
|
|
new BinlogOffset("mysql-bin.000001", 0L),
|
|
|
|
|
finishedSplitsInfo,
|
|
|
|
|
databaseHistory);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private Tuple2<String, BinlogPosition> getTestSplitInfo(TableId tableId, int splitNo) {
|
|
|
|
|
private Tuple2<String, BinlogOffset> getTestSplitInfo(TableId tableId, int splitNo) {
|
|
|
|
|
final String splitId = tableId.toString() + "-" + splitNo;
|
|
|
|
|
final BinlogPosition highWatermark =
|
|
|
|
|
new BinlogPosition("mysql-bin.000001", 0L + splitNo * 200);
|
|
|
|
|
final BinlogOffset highWatermark = new BinlogOffset("mysql-bin.000001", 0L + splitNo * 200);
|
|
|
|
|
return Tuple2.of(splitId, highWatermark);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -202,7 +201,7 @@ public class MySQLParallelSourceEnumStateSerializerTest {
|
|
|
|
|
assertEquals(
|
|
|
|
|
expected.getFinishedSnapshotSplits().size(),
|
|
|
|
|
actual.getFinishedSnapshotSplits().size());
|
|
|
|
|
for (Map.Entry<Integer, List<Tuple2<String, BinlogPosition>>> entry :
|
|
|
|
|
for (Map.Entry<Integer, List<Tuple2<String, BinlogOffset>>> entry :
|
|
|
|
|
expected.getFinishedSnapshotSplits().entrySet()) {
|
|
|
|
|
assertEquals(
|
|
|
|
|
entry.getValue().toString(),
|
|
|
|
|