|
|
|
@ -34,16 +34,25 @@ import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.testutils.RecordsFormatter;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase;
|
|
|
|
|
import io.debezium.connector.mysql.MySqlConnection;
|
|
|
|
|
import io.debezium.data.Envelope;
|
|
|
|
|
import io.debezium.jdbc.JdbcConnection;
|
|
|
|
|
import io.debezium.pipeline.EventDispatcher;
|
|
|
|
|
import io.debezium.pipeline.spi.OffsetContext;
|
|
|
|
|
import io.debezium.relational.TableId;
|
|
|
|
|
import io.debezium.schema.DataCollectionSchema;
|
|
|
|
|
import org.apache.kafka.connect.data.Struct;
|
|
|
|
|
import org.apache.kafka.connect.header.ConnectHeaders;
|
|
|
|
|
import org.apache.kafka.connect.source.SourceRecord;
|
|
|
|
|
import org.junit.BeforeClass;
|
|
|
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
|
|
import java.sql.SQLException;
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
import java.util.Arrays;
|
|
|
|
|
import java.util.Iterator;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.Optional;
|
|
|
|
|
import java.util.function.Supplier;
|
|
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
|
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
@ -69,6 +78,8 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
@Test
|
|
|
|
|
public void testReadSingleSnapshotSplit() throws Exception {
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {"customers_even_dist"}, 4);
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
|
DataTypes.FIELD("id", DataTypes.BIGINT()),
|
|
|
|
@ -84,13 +95,16 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
"+I[103, user_3, Shanghai, 123567891234]",
|
|
|
|
|
"+I[104, user_4, Shanghai, 123567891234]"
|
|
|
|
|
};
|
|
|
|
|
List<String> actual = readTableSnapshotSplits(mySqlSplits, sourceConfig, 1, dataType);
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, statefulTaskContext, 1, dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testReadAllSnapshotSplitsForOneTable() throws Exception {
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {"customers_even_dist"}, 4);
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
@ -114,13 +128,16 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
"+I[110, user_10, Shanghai, 123567891234]"
|
|
|
|
|
};
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, sourceConfig, mySqlSplits.size(), dataType);
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testReadAllSplitForTableWithSingleLine() throws Exception {
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {"customer_card_single_line"}, 10);
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
@ -131,7 +148,8 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
List<MySqlSplit> mySqlSplits = getMySqlSplits(sourceConfig);
|
|
|
|
|
String[] expected = new String[] {"+I[20001, LEVEL_1, user_1, user with level 1]"};
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, sourceConfig, mySqlSplits.size(), dataType);
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -139,6 +157,8 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
public void testReadAllSnapshotSplitsForTables() throws Exception {
|
|
|
|
|
MySqlSourceConfig sourceConfig =
|
|
|
|
|
getConfig(new String[] {"customer_card", "customer_card_single_line"}, 10);
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
|
|
|
|
|
DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
@ -172,7 +192,8 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
"+I[50003, LEVEL_1, user_14, user with level 1]"
|
|
|
|
|
};
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, sourceConfig, mySqlSplits.size(), dataType);
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -180,6 +201,8 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
public void testThrowRuntimeExceptionInSnapshotScan() throws Exception {
|
|
|
|
|
MySqlSourceConfig sourceConfig =
|
|
|
|
|
getConfig(new String[] {"customer_card", "customers_1"}, 10);
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
|
|
|
|
|
DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
@ -196,7 +219,7 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
|
|
|
|
|
String exceptionMessage = String.format("Snapshotting of table %s failed.", tableToDrop);
|
|
|
|
|
try {
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, sourceConfig, mySqlSplits.size(), dataType);
|
|
|
|
|
readTableSnapshotSplits(mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
fail("Should fail.");
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
assertTrue(e instanceof FlinkRuntimeException);
|
|
|
|
@ -204,15 +227,169 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testChangingDataInSnapshotScan() throws Exception {
|
|
|
|
|
String tableName = "customers_even_dist";
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {tableName}, 10);
|
|
|
|
|
|
|
|
|
|
String tableId = customerDatabase.getDatabaseName() + "." + tableName;
|
|
|
|
|
String[] changingDataSql =
|
|
|
|
|
new String[] {
|
|
|
|
|
"UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103",
|
|
|
|
|
"DELETE FROM " + tableId + " where id = 102",
|
|
|
|
|
"INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')",
|
|
|
|
|
"UPDATE " + tableId + " SET address = 'Shanghai' where id = 103"
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new MakeBinlogEventTaskContext(
|
|
|
|
|
sourceConfig,
|
|
|
|
|
binaryLogClient,
|
|
|
|
|
mySqlConnection,
|
|
|
|
|
() -> executeSql(sourceConfig, changingDataSql));
|
|
|
|
|
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
|
DataTypes.FIELD("id", DataTypes.BIGINT()),
|
|
|
|
|
DataTypes.FIELD("name", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("address", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("phone_number", DataTypes.STRING()));
|
|
|
|
|
List<MySqlSplit> mySqlSplits = getMySqlSplits(sourceConfig);
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
new String[] {
|
|
|
|
|
"+I[101, user_1, Shanghai, 123567891234]",
|
|
|
|
|
"+I[102, user_2, Shanghai, 123567891234]",
|
|
|
|
|
"+I[103, user_3, Shanghai, 123567891234]",
|
|
|
|
|
"+I[104, user_4, Shanghai, 123567891234]",
|
|
|
|
|
"+I[105, user_5, Shanghai, 123567891234]",
|
|
|
|
|
"+I[106, user_6, Shanghai, 123567891234]",
|
|
|
|
|
"+I[107, user_7, Shanghai, 123567891234]",
|
|
|
|
|
"+I[108, user_8, Shanghai, 123567891234]",
|
|
|
|
|
"+I[109, user_9, Shanghai, 123567891234]",
|
|
|
|
|
"+I[110, user_10, Shanghai, 123567891234]"
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testInsertDataInSnapshotScan() throws Exception {
|
|
|
|
|
String tableName = "customers_even_dist";
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {tableName}, 10);
|
|
|
|
|
|
|
|
|
|
String tableId = customerDatabase.getDatabaseName() + "." + tableName;
|
|
|
|
|
String[] insertDataSql =
|
|
|
|
|
new String[] {
|
|
|
|
|
"INSERT INTO " + tableId + " VALUES(111, 'user_11','Shanghai','123567891234')",
|
|
|
|
|
"INSERT INTO " + tableId + " VALUES(112, 'user_12','Shanghai','123567891234')",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
String[] recoveryDataSql =
|
|
|
|
|
new String[] {
|
|
|
|
|
"DELETE FROM " + tableId + " where id = 111",
|
|
|
|
|
"DELETE FROM " + tableId + " where id = 112",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new MakeBinlogEventTaskContext(
|
|
|
|
|
sourceConfig,
|
|
|
|
|
binaryLogClient,
|
|
|
|
|
mySqlConnection,
|
|
|
|
|
() -> executeSql(sourceConfig, insertDataSql));
|
|
|
|
|
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
|
DataTypes.FIELD("id", DataTypes.BIGINT()),
|
|
|
|
|
DataTypes.FIELD("name", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("address", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("phone_number", DataTypes.STRING()));
|
|
|
|
|
List<MySqlSplit> mySqlSplits = getMySqlSplits(sourceConfig);
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
new String[] {
|
|
|
|
|
"+I[101, user_1, Shanghai, 123567891234]",
|
|
|
|
|
"+I[102, user_2, Shanghai, 123567891234]",
|
|
|
|
|
"+I[103, user_3, Shanghai, 123567891234]",
|
|
|
|
|
"+I[104, user_4, Shanghai, 123567891234]",
|
|
|
|
|
"+I[105, user_5, Shanghai, 123567891234]",
|
|
|
|
|
"+I[106, user_6, Shanghai, 123567891234]",
|
|
|
|
|
"+I[107, user_7, Shanghai, 123567891234]",
|
|
|
|
|
"+I[108, user_8, Shanghai, 123567891234]",
|
|
|
|
|
"+I[109, user_9, Shanghai, 123567891234]",
|
|
|
|
|
"+I[110, user_10, Shanghai, 123567891234]",
|
|
|
|
|
"+I[111, user_11, Shanghai, 123567891234]",
|
|
|
|
|
"+I[112, user_12, Shanghai, 123567891234]"
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
executeSql(sourceConfig, recoveryDataSql);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testDeleteDataInSnapshotScan() throws Exception {
|
|
|
|
|
String tableName = "customers_even_dist";
|
|
|
|
|
MySqlSourceConfig sourceConfig = getConfig(new String[] {tableName}, 10);
|
|
|
|
|
|
|
|
|
|
String tableId = customerDatabase.getDatabaseName() + "." + tableName;
|
|
|
|
|
String[] deleteDataSql =
|
|
|
|
|
new String[] {
|
|
|
|
|
"DELETE FROM " + tableId + " where id = 101",
|
|
|
|
|
"DELETE FROM " + tableId + " where id = 102",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
String[] recoveryDataSql =
|
|
|
|
|
new String[] {
|
|
|
|
|
"INSERT INTO " + tableId + " VALUES(101, 'user_1','Shanghai','123567891234')",
|
|
|
|
|
"INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new MakeBinlogEventTaskContext(
|
|
|
|
|
sourceConfig,
|
|
|
|
|
binaryLogClient,
|
|
|
|
|
mySqlConnection,
|
|
|
|
|
() -> executeSql(sourceConfig, deleteDataSql));
|
|
|
|
|
|
|
|
|
|
final DataType dataType =
|
|
|
|
|
DataTypes.ROW(
|
|
|
|
|
DataTypes.FIELD("id", DataTypes.BIGINT()),
|
|
|
|
|
DataTypes.FIELD("name", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("address", DataTypes.STRING()),
|
|
|
|
|
DataTypes.FIELD("phone_number", DataTypes.STRING()));
|
|
|
|
|
List<MySqlSplit> mySqlSplits = getMySqlSplits(sourceConfig);
|
|
|
|
|
|
|
|
|
|
String[] expected =
|
|
|
|
|
new String[] {
|
|
|
|
|
"+I[103, user_3, Shanghai, 123567891234]",
|
|
|
|
|
"+I[104, user_4, Shanghai, 123567891234]",
|
|
|
|
|
"+I[105, user_5, Shanghai, 123567891234]",
|
|
|
|
|
"+I[106, user_6, Shanghai, 123567891234]",
|
|
|
|
|
"+I[107, user_7, Shanghai, 123567891234]",
|
|
|
|
|
"+I[108, user_8, Shanghai, 123567891234]",
|
|
|
|
|
"+I[109, user_9, Shanghai, 123567891234]",
|
|
|
|
|
"+I[110, user_10, Shanghai, 123567891234]",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
List<String> actual =
|
|
|
|
|
readTableSnapshotSplits(
|
|
|
|
|
mySqlSplits, statefulTaskContext, mySqlSplits.size(), dataType);
|
|
|
|
|
assertEqualsInAnyOrder(Arrays.asList(expected), actual);
|
|
|
|
|
executeSql(sourceConfig, recoveryDataSql);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<String> readTableSnapshotSplits(
|
|
|
|
|
List<MySqlSplit> mySqlSplits,
|
|
|
|
|
MySqlSourceConfig sourceConfig,
|
|
|
|
|
StatefulTaskContext statefulTaskContext,
|
|
|
|
|
int scanSplitsNum,
|
|
|
|
|
DataType dataType)
|
|
|
|
|
throws Exception {
|
|
|
|
|
|
|
|
|
|
StatefulTaskContext statefulTaskContext =
|
|
|
|
|
new StatefulTaskContext(sourceConfig, binaryLogClient, mySqlConnection);
|
|
|
|
|
SnapshotSplitReader snapshotSplitReader = new SnapshotSplitReader(statefulTaskContext, 0);
|
|
|
|
|
|
|
|
|
|
List<SourceRecord> result = new ArrayList<>();
|
|
|
|
@ -284,4 +461,57 @@ public class SnapshotSplitReaderTest extends MySqlSourceTestBase {
|
|
|
|
|
.password(customerDatabase.getPassword())
|
|
|
|
|
.createConfig(0);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private boolean executeSql(MySqlSourceConfig sourceConfig, String[] sqlStatements) {
|
|
|
|
|
JdbcConnection connection = DebeziumUtils.openJdbcConnection(sourceConfig);
|
|
|
|
|
try {
|
|
|
|
|
connection.setAutoCommit(false);
|
|
|
|
|
connection.execute(sqlStatements);
|
|
|
|
|
connection.commit();
|
|
|
|
|
} catch (SQLException e) {
|
|
|
|
|
LOG.error("Failed to execute sql statements.", e);
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class MakeBinlogEventTaskContext extends StatefulTaskContext {
|
|
|
|
|
|
|
|
|
|
private Supplier<Boolean> makeBinlogFunction;
|
|
|
|
|
|
|
|
|
|
public MakeBinlogEventTaskContext(
|
|
|
|
|
MySqlSourceConfig sourceConfig,
|
|
|
|
|
BinaryLogClient binaryLogClient,
|
|
|
|
|
MySqlConnection connection,
|
|
|
|
|
Supplier<Boolean> makeBinlogFunction) {
|
|
|
|
|
super(sourceConfig, binaryLogClient, connection);
|
|
|
|
|
this.makeBinlogFunction = makeBinlogFunction;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public EventDispatcher.SnapshotReceiver getSnapshotReceiver() {
|
|
|
|
|
EventDispatcher.SnapshotReceiver snapshotReceiver = super.getSnapshotReceiver();
|
|
|
|
|
return new EventDispatcher.SnapshotReceiver() {
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void changeRecord(
|
|
|
|
|
DataCollectionSchema schema,
|
|
|
|
|
Envelope.Operation operation,
|
|
|
|
|
Object key,
|
|
|
|
|
Struct value,
|
|
|
|
|
OffsetContext offset,
|
|
|
|
|
ConnectHeaders headers)
|
|
|
|
|
throws InterruptedException {
|
|
|
|
|
snapshotReceiver.changeRecord(schema, operation, key, value, offset, headers);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void completeSnapshot() throws InterruptedException {
|
|
|
|
|
snapshotReceiver.completeSnapshot();
|
|
|
|
|
// make binlog events
|
|
|
|
|
makeBinlogFunction.get();
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|