@@ -340,4 +340,101 @@ void testSerializeComplexTypes() throws Exception {
340340 assertThat (rowNode .has ("f1" )).isTrue ();
341341 assertThat (rowNode .has ("f2" )).isTrue ();
342342 }
343+
344+ @ Test
345+ void testSerializeWithSchemaComplexTypes () throws Exception {
346+ ObjectMapper mapper =
347+ JacksonMapperFactory .createObjectMapper ()
348+ .configure (JsonGenerator .Feature .WRITE_BIGDECIMAL_AS_PLAIN , false );
349+ Map <String , String > properties = new HashMap <>();
350+ properties .put ("include-schema.enabled" , "true" );
351+ Configuration configuration = Configuration .fromMap (properties );
352+ SerializationSchema <Event > serializationSchema =
353+ ChangeLogJsonFormatFactory .createSerializationSchema (
354+ configuration , JsonSerializationType .DEBEZIUM_JSON , ZoneId .systemDefault ());
355+ serializationSchema .open (new MockInitializationContext ());
356+
357+ // create table with complex types
358+ Schema schema =
359+ Schema .newBuilder ()
360+ .physicalColumn ("id" , DataTypes .INT ())
361+ .physicalColumn ("arr" , DataTypes .ARRAY (DataTypes .STRING ()))
362+ .physicalColumn ("map" , DataTypes .MAP (DataTypes .STRING (), DataTypes .INT ()))
363+ .physicalColumn (
364+ "row" ,
365+ DataTypes .ROW (
366+ DataTypes .FIELD ("f1" , DataTypes .STRING ()),
367+ DataTypes .FIELD ("f2" , DataTypes .INT ())))
368+ .primaryKey ("id" )
369+ .build ();
370+
371+ RowType rowType =
372+ RowType .of (
373+ DataTypes .INT (),
374+ DataTypes .ARRAY (DataTypes .STRING ()),
375+ DataTypes .MAP (DataTypes .STRING (), DataTypes .INT ()),
376+ DataTypes .ROW (
377+ DataTypes .FIELD ("f1" , DataTypes .STRING ()),
378+ DataTypes .FIELD ("f2" , DataTypes .INT ())));
379+
380+ CreateTableEvent createTableEvent = new CreateTableEvent (TABLE_1 , schema );
381+ assertThat (serializationSchema .serialize (createTableEvent )).isNull ();
382+
383+ BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator (rowType );
384+
385+ // Create test data with complex types
386+ org .apache .flink .cdc .common .data .GenericArrayData arrayData =
387+ new org .apache .flink .cdc .common .data .GenericArrayData (
388+ new Object [] {
389+ BinaryStringData .fromString ("item1" ),
390+ BinaryStringData .fromString ("item2" )
391+ });
392+
393+ Map <Object , Object > mapValues = new HashMap <>();
394+ mapValues .put (BinaryStringData .fromString ("key1" ), 100 );
395+ mapValues .put (BinaryStringData .fromString ("key2" ), 200 );
396+ org .apache .flink .cdc .common .data .GenericMapData mapData =
397+ new org .apache .flink .cdc .common .data .GenericMapData (mapValues );
398+
399+ BinaryRecordDataGenerator nestedRowGenerator =
400+ new BinaryRecordDataGenerator (RowType .of (DataTypes .STRING (), DataTypes .INT ()));
401+ org .apache .flink .cdc .common .data .RecordData nestedRow =
402+ nestedRowGenerator .generate (
403+ new Object [] {BinaryStringData .fromString ("nested" ), 42 });
404+
405+ // insert event with complex types
406+ DataChangeEvent insertEvent =
407+ DataChangeEvent .insertEvent (
408+ TABLE_1 ,
409+ generator .generate (new Object [] {1 , arrayData , mapData , nestedRow }));
410+
411+ byte [] serialized = serializationSchema .serialize (insertEvent );
412+ JsonNode actual = mapper .readTree (serialized );
413+
414+ JsonNode expected =
415+ mapper .readTree (
416+ "{\" schema\" :{\" type\" :\" struct\" ,\" fields\" :["
417+ + "{\" type\" :\" struct\" ,\" fields\" :["
418+ + "{\" type\" :\" int32\" ,\" optional\" :true,\" field\" :\" id\" },"
419+ + "{\" type\" :\" array\" ,\" items\" :{\" type\" :\" string\" ,\" optional\" :false},\" optional\" :true,\" field\" :\" arr\" },"
420+ + "{\" type\" :\" map\" ,\" keys\" :{\" type\" :\" string\" ,\" optional\" :false},\" values\" :{\" type\" :\" int32\" ,\" optional\" :false},\" optional\" :true,\" field\" :\" map\" },"
421+ + "{\" type\" :\" struct\" ,\" fields\" :["
422+ + "{\" type\" :\" string\" ,\" optional\" :false,\" field\" :\" f1\" },"
423+ + "{\" type\" :\" int32\" ,\" optional\" :false,\" field\" :\" f2\" }"
424+ + "],\" optional\" :true,\" field\" :\" row\" }"
425+ + "],\" optional\" :true,\" field\" :\" before\" },"
426+ + "{\" type\" :\" struct\" ,\" fields\" :["
427+ + "{\" type\" :\" int32\" ,\" optional\" :true,\" field\" :\" id\" },"
428+ + "{\" type\" :\" array\" ,\" items\" :{\" type\" :\" string\" ,\" optional\" :false},\" optional\" :true,\" field\" :\" arr\" },"
429+ + "{\" type\" :\" map\" ,\" keys\" :{\" type\" :\" string\" ,\" optional\" :false},\" values\" :{\" type\" :\" int32\" ,\" optional\" :false},\" optional\" :true,\" field\" :\" map\" },"
430+ + "{\" type\" :\" struct\" ,\" fields\" :["
431+ + "{\" type\" :\" string\" ,\" optional\" :false,\" field\" :\" f1\" },"
432+ + "{\" type\" :\" int32\" ,\" optional\" :false,\" field\" :\" f2\" }"
433+ + "],\" optional\" :true,\" field\" :\" row\" }"
434+ + "],\" optional\" :true,\" field\" :\" after\" }"
435+ + "],\" optional\" :false},"
436+ + "\" payload\" :{\" before\" :null,\" after\" :{\" id\" :1,\" arr\" :[\" item1\" ,\" item2\" ],\" map\" :{\" key1\" :100,\" key2\" :200},\" row\" :{\" f1\" :\" nested\" ,\" f2\" :42}},\" op\" :\" c\" ,\" source\" :{\" db\" :\" default_schema\" ,\" table\" :\" table1\" }}}" );
437+
438+ assertThat (actual ).isEqualTo (expected );
439+ }
343440}
0 commit comments