67
67
public class MongoDBParallelSourceITCase extends MongoDBSourceTestBase {
68
68
private static final int USE_POST_LOWWATERMARK_HOOK = 1 ;
69
69
private static final int USE_PRE_HIGHWATERMARK_HOOK = 2 ;
70
+ private static final int USE_POST_HIGHWATERMARK_HOOK = 3 ;
70
71
71
72
@ Rule public final Timeout timeoutPerTest = Timeout .seconds (300 );
72
73
@@ -150,8 +151,7 @@ public void testReadSingleTableWithSingleParallelismAndSkipBackfill() throws Exc
150
151
@ Test
151
152
public void testEnableBackfillWithDMLPreHighWaterMark () throws Exception {
152
153
153
- List <String > records =
154
- testBackfillWhenWritingEvents (false , 21 , USE_PRE_HIGHWATERMARK_HOOK , false );
154
+ List <String > records = testBackfillWhenWritingEvents (false , 21 , USE_PRE_HIGHWATERMARK_HOOK );
155
155
156
156
List <String > expectedRecords =
157
157
Arrays .asList (
@@ -184,8 +184,7 @@ public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception {
184
184
@ Test
185
185
public void testEnableBackfillWithDMLPostLowWaterMark () throws Exception {
186
186
187
- List <String > records =
188
- testBackfillWhenWritingEvents (false , 21 , USE_POST_LOWWATERMARK_HOOK , false );
187
+ List <String > records = testBackfillWhenWritingEvents (false , 21 , USE_POST_LOWWATERMARK_HOOK );
189
188
190
189
List <String > expectedRecords =
191
190
Arrays .asList (
@@ -216,10 +215,45 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception {
216
215
}
217
216
218
217
@ Test
219
- public void testSkipBackfillWithDMLPreHighWaterMark () throws Exception {
218
+ public void testEnableBackfillWithDMLPostHighWaterMark () throws Exception {
220
219
221
220
List <String > records =
222
- testBackfillWhenWritingEvents (true , 24 , USE_PRE_HIGHWATERMARK_HOOK , false );
221
+ testBackfillWhenWritingEvents (false , 24 , USE_POST_HIGHWATERMARK_HOOK );
222
+ List <String > expectedRecords =
223
+ Arrays .asList (
224
+ "+I[101, user_1, Shanghai, 123567891234]" ,
225
+ "+I[102, user_2, Shanghai, 123567891234]" ,
226
+ "+I[103, user_3, Shanghai, 123567891234]" ,
227
+ "+I[109, user_4, Shanghai, 123567891234]" ,
228
+ "+I[110, user_5, Shanghai, 123567891234]" ,
229
+ "+I[111, user_6, Shanghai, 123567891234]" ,
230
+ "+I[118, user_7, Shanghai, 123567891234]" ,
231
+ "+I[121, user_8, Shanghai, 123567891234]" ,
232
+ "+I[123, user_9, Shanghai, 123567891234]" ,
233
+ "+I[1009, user_10, Shanghai, 123567891234]" ,
234
+ "+I[1010, user_11, Shanghai, 123567891234]" ,
235
+ "+I[1011, user_12, Shanghai, 123567891234]" ,
236
+ "+I[1012, user_13, Shanghai, 123567891234]" ,
237
+ "+I[1013, user_14, Shanghai, 123567891234]" ,
238
+ "+I[1014, user_15, Shanghai, 123567891234]" ,
239
+ "+I[1015, user_16, Shanghai, 123567891234]" ,
240
+ "+I[1016, user_17, Shanghai, 123567891234]" ,
241
+ "+I[1017, user_18, Shanghai, 123567891234]" ,
242
+ "+I[1018, user_19, Shanghai, 123567891234]" ,
243
+ "+I[1019, user_20, Shanghai, 123567891234]" ,
244
+ "+I[2000, user_21, Shanghai, 123567891234]" ,
245
+ "+I[15213, user_15213, Shanghai, 123567891234]" ,
246
+ "+U[2000, user_21, Pittsburgh, 123567891234]" ,
247
+ // delete message only contains _id, sql job contain value because of
248
+ // changelog normalization
249
+ "-D[0, null, null, null]" );
250
+ assertEqualsInAnyOrder (expectedRecords , records );
251
+ }
252
+
253
+ @ Test
254
+ public void testSkipBackfillWithDMLPreHighWaterMark () throws Exception {
255
+
256
+ List <String > records = testBackfillWhenWritingEvents (true , 24 , USE_PRE_HIGHWATERMARK_HOOK );
223
257
224
258
List <String > expectedRecords =
225
259
Arrays .asList (
@@ -257,8 +291,7 @@ public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception {
257
291
@ Test
258
292
public void testSkipBackfillWithDMLPostLowWaterMark () throws Exception {
259
293
260
- List <String > records =
261
- testBackfillWhenWritingEvents (true , 24 , USE_POST_LOWWATERMARK_HOOK , false );
294
+ List <String > records = testBackfillWhenWritingEvents (true , 24 , USE_POST_LOWWATERMARK_HOOK );
262
295
263
296
List <String > expectedRecords =
264
297
Arrays .asList (
@@ -295,8 +328,7 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception {
295
328
}
296
329
297
330
private List <String > testBackfillWhenWritingEvents (
298
- boolean skipBackFill , int fetchSize , int hookType , boolean enableFullDocPrePostImage )
299
- throws Exception {
331
+ boolean skipBackFill , int fetchSize , int hookType ) throws Exception {
300
332
String customerDatabase = CONTAINER .executeCommandFileInSeparateDatabase ("customer" );
301
333
StreamExecutionEnvironment env = StreamExecutionEnvironment .getExecutionEnvironment ();
302
334
env .enableCheckpointing (1000 );
@@ -319,11 +351,11 @@ private List<String> testBackfillWhenWritingEvents(
319
351
.username (FLINK_USER )
320
352
.password (FLINK_USER_PASSWORD )
321
353
.startupOptions (StartupOptions .initial ())
322
- .scanFullChangelog (enableFullDocPrePostImage )
354
+ .scanFullChangelog (false )
323
355
.collectionList (
324
356
getCollectionNameRegex (
325
357
customerDatabase , new String [] {"customers" }))
326
- .deserializer (customerTable .getDeserializer (enableFullDocPrePostImage ))
358
+ .deserializer (customerTable .getDeserializer (false ))
327
359
.skipSnapshotBackfill (skipBackFill )
328
360
.build ();
329
361
@@ -347,10 +379,16 @@ private List<String> testBackfillWhenWritingEvents(
347
379
mongoCollection .deleteOne (Filters .eq ("cid" , 1019L ));
348
380
};
349
381
350
- if (hookType == USE_POST_LOWWATERMARK_HOOK ) {
351
- hooks .setPostLowWatermarkAction (snapshotPhaseHook );
352
- } else if (hookType == USE_PRE_HIGHWATERMARK_HOOK ) {
353
- hooks .setPreHighWatermarkAction (snapshotPhaseHook );
382
+ switch (hookType ) {
383
+ case USE_POST_LOWWATERMARK_HOOK :
384
+ hooks .setPostLowWatermarkAction (snapshotPhaseHook );
385
+ break ;
386
+ case USE_PRE_HIGHWATERMARK_HOOK :
387
+ hooks .setPreHighWatermarkAction (snapshotPhaseHook );
388
+ break ;
389
+ case USE_POST_HIGHWATERMARK_HOOK :
390
+ hooks .setPostHighWatermarkAction (snapshotPhaseHook );
391
+ break ;
354
392
}
355
393
source .setSnapshotHooks (hooks );
356
394
0 commit comments