@ -55,6 +55,7 @@ import java.util.Collections;
import java.util.List ;
import java.util.Random ;
import java.util.stream.Collectors ;
import java.util.stream.Stream ;
import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBAssertUtils.assertEqualsInAnyOrder ;
import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER ;
@ -590,6 +591,148 @@ public class MongoDBFullChangelogITCase extends MongoDBSourceTestBase {
return records ;
}
@Test
public void testMetadataColumns ( ) throws Exception {
testMongoDBParallelSourceWithMetadataColumns (
DEFAULT_PARALLELISM , new String [ ] { "customers" } , true ) ;
}
private void testMongoDBParallelSourceWithMetadataColumns (
int parallelism , String [ ] captureCustomerCollections , boolean skipSnapshotBackfill )
throws Exception {
String customerDatabase =
"customer_" + Integer . toUnsignedString ( new Random ( ) . nextInt ( ) , 36 ) ;
// A - enable system-level fulldoc pre & post image feature
mongoContainer . executeCommand (
"use admin; db.runCommand({ setClusterParameter: { changeStreamOptions: { preAndPostImages: { expireAfterSeconds: 'off' } } } })" ) ;
// B - enable collection-level fulldoc pre & post image for change capture collection
for ( String collectionName : captureCustomerCollections ) {
mongoContainer . executeCommandInDatabase (
String . format (
"db.createCollection('%s'); db.runCommand({ collMod: '%s', changeStreamPreAndPostImages: { enabled: true } })" ,
collectionName , collectionName ) ,
customerDatabase ) ;
}
StreamExecutionEnvironment env = StreamExecutionEnvironment . getExecutionEnvironment ( ) ;
StreamTableEnvironment tEnv = StreamTableEnvironment . create ( env ) ;
env . setParallelism ( parallelism ) ;
env . enableCheckpointing ( 200L ) ;
env . setRestartStrategy ( RestartStrategies . fixedDelayRestart ( 1 , 0 ) ) ;
String sourceDDL =
String . format (
"CREATE TABLE customers ("
+ " _id STRING NOT NULL,"
+ " cid BIGINT NOT NULL,"
+ " name STRING,"
+ " address STRING,"
+ " phone_number STRING,"
+ " database_name STRING METADATA VIRTUAL,"
+ " collection_name STRING METADATA VIRTUAL,"
+ " row_kind STRING METADATA VIRTUAL,"
+ " primary key (_id) not enforced"
+ ") WITH ("
+ " 'connector' = 'mongodb-cdc',"
+ " 'scan.incremental.snapshot.enabled' = '%s',"
+ " 'hosts' = '%s',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database' = '%s',"
+ " 'collection' = '%s',"
+ " 'heartbeat.interval.ms' = '500',"
+ " 'scan.full-changelog' = 'true',"
+ " 'scan.incremental.snapshot.backfill.skip' = '%s'"
+ ")" ,
parallelismSnapshot ? "true" : "false" ,
mongoContainer . getHostAndPort ( ) ,
FLINK_USER ,
FLINK_USER_PASSWORD ,
customerDatabase ,
getCollectionNameRegex ( customerDatabase , captureCustomerCollections ) ,
skipSnapshotBackfill ) ;
mongoContainer . executeCommandFileInDatabase ( "customer" , customerDatabase ) ;
// first step: check the snapshot data
List < String > snapshotForSingleTable =
Stream . of (
"+I[%s, %s, +I, 101, user_1, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 102, user_2, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 103, user_3, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 109, user_4, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 110, user_5, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 111, user_6, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 118, user_7, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 121, user_8, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 123, user_9, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1009, user_10, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1010, user_11, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1011, user_12, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1012, user_13, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1013, user_14, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1014, user_15, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1015, user_16, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1016, user_17, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1017, user_18, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1018, user_19, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 1019, user_20, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 2000, user_21, Shanghai, 123567891234]" )
. map ( s - > String . format ( s , customerDatabase , captureCustomerCollections [ 0 ] ) )
. collect ( Collectors . toList ( ) ) ;
tEnv . executeSql ( sourceDDL ) ;
TableResult tableResult =
tEnv . executeSql (
"select database_name, collection_name, row_kind, "
+ "cid, name, address, phone_number from customers" ) ;
CloseableIterator < Row > iterator = tableResult . collect ( ) ;
JobID jobId = tableResult . getJobClient ( ) . get ( ) . getJobID ( ) ;
List < String > expectedSnapshotData = new ArrayList < > ( ) ;
for ( int i = 0 ; i < captureCustomerCollections . length ; i + + ) {
expectedSnapshotData . addAll ( snapshotForSingleTable ) ;
}
assertEqualsInAnyOrder (
expectedSnapshotData , fetchRows ( iterator , expectedSnapshotData . size ( ) ) ) ;
// second step: check the change stream data
for ( String collectionName : captureCustomerCollections ) {
makeFirstPartChangeStreamEvents (
mongodbClient . getDatabase ( customerDatabase ) , collectionName ) ;
}
for ( String collectionName : captureCustomerCollections ) {
makeSecondPartChangeStreamEvents (
mongodbClient . getDatabase ( customerDatabase ) , collectionName ) ;
}
List < String > changeEventsForSingleTable =
Stream . of (
"-U[%s, %s, -U, 101, user_1, Shanghai, 123567891234]" ,
"+U[%s, %s, +U, 101, user_1, Hangzhou, 123567891234]" ,
"-D[%s, %s, -D, 102, user_2, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 102, user_2, Shanghai, 123567891234]" ,
"-U[%s, %s, -U, 103, user_3, Shanghai, 123567891234]" ,
"+U[%s, %s, +U, 103, user_3, Hangzhou, 123567891234]" ,
"-U[%s, %s, -U, 1010, user_11, Shanghai, 123567891234]" ,
"+U[%s, %s, +U, 1010, user_11, Hangzhou, 123567891234]" ,
"+I[%s, %s, +I, 2001, user_22, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 2002, user_23, Shanghai, 123567891234]" ,
"+I[%s, %s, +I, 2003, user_24, Shanghai, 123567891234]" )
. map ( s - > String . format ( s , customerDatabase , captureCustomerCollections [ 0 ] ) )
. collect ( Collectors . toList ( ) ) ;
List < String > expectedChangeStreamData = new ArrayList < > ( ) ;
for ( int i = 0 ; i < captureCustomerCollections . length ; i + + ) {
expectedChangeStreamData . addAll ( changeEventsForSingleTable ) ;
}
List < String > actualChangeStreamData = fetchRows ( iterator , expectedChangeStreamData . size ( ) ) ;
assertEqualsInAnyOrder ( expectedChangeStreamData , actualChangeStreamData ) ;
tableResult . getJobClient ( ) . get ( ) . cancel ( ) . get ( ) ;
}
private void testMongoDBParallelSource (
MongoDBTestUtils . FailoverType failoverType ,
MongoDBTestUtils . FailoverPhase failoverPhase ,