30
30
import org .apache .flink .cdc .common .schema .Schema ;
31
31
import org .apache .flink .cdc .common .types .DataTypes ;
32
32
import org .apache .flink .cdc .common .types .RowType ;
33
+ import org .apache .flink .cdc .common .utils .SchemaUtils ;
33
34
import org .apache .flink .cdc .runtime .typeutils .BinaryRecordDataGenerator ;
34
35
35
36
import java .util .ArrayList ;
@@ -54,7 +55,8 @@ public enum EventSetId {
54
55
SINGLE_SPLIT_MULTI_TABLES ,
55
56
MULTI_SPLITS_SINGLE_TABLE ,
56
57
CUSTOM_SOURCE_EVENTS ,
57
- TRANSFORM_TABLE
58
+ TRANSFORM_TABLE ,
59
+ COMPLEX_COLUMN_NAME_TABLE
58
60
}
59
61
60
62
public static final TableId TABLE_1 =
@@ -120,6 +122,11 @@ public static void setSourceEvents(EventSetId eventType) {
120
122
sourceEvents = transformTable ();
121
123
break ;
122
124
}
125
+ case COMPLEX_COLUMN_NAME_TABLE :
126
+ {
127
+ sourceEvents = complexColumnNameTable ();
128
+ break ;
129
+ }
123
130
default :
124
131
throw new IllegalArgumentException (eventType + " is not supported" );
125
132
}
@@ -644,4 +651,126 @@ public static List<List<Event>> transformTable() {
644
651
eventOfSplits .add (split1 );
645
652
return eventOfSplits ;
646
653
}
654
+
655
+ public static List <List <Event >> complexColumnNameTable () {
656
+ List <List <Event >> eventOfSplits = new ArrayList <>();
657
+ List <Event > split1 = new ArrayList <>();
658
+
659
+ // create table
660
+ Schema schema =
661
+ Schema .newBuilder ()
662
+ .physicalColumn ("class" , DataTypes .STRING ())
663
+ .physicalColumn ("foo-bar" , DataTypes .INT ())
664
+ .physicalColumn ("bar-foo" , DataTypes .INT ())
665
+ .primaryKey ("class" )
666
+ .build ();
667
+ CreateTableEvent createTableEvent = new CreateTableEvent (TABLE_1 , schema );
668
+ split1 .add (createTableEvent );
669
+
670
+ BinaryRecordDataGenerator generator =
671
+ new BinaryRecordDataGenerator ((RowType ) schema .toRowDataType ());
672
+ // insert
673
+ DataChangeEvent insertEvent1 =
674
+ DataChangeEvent .insertEvent (
675
+ TABLE_1 ,
676
+ generator .generate (
677
+ new Object [] {
678
+ BinaryStringData .fromString ("class0" ), 0 , 0 ,
679
+ }),
680
+ new HashMap <String , String >() {
681
+ {
682
+ put ("timestamp-type" , "type0" );
683
+ }
684
+ });
685
+ split1 .add (insertEvent1 );
686
+ DataChangeEvent insertEvent2 =
687
+ DataChangeEvent .insertEvent (
688
+ TABLE_1 ,
689
+ generator .generate (
690
+ new Object [] {
691
+ BinaryStringData .fromString ("class1" ), 1 , 10 ,
692
+ }),
693
+ new HashMap <String , String >() {
694
+ {
695
+ put ("timestamp-type" , "type1" );
696
+ }
697
+ });
698
+ split1 .add (insertEvent2 );
699
+ DataChangeEvent insertEvent3 =
700
+ DataChangeEvent .insertEvent (
701
+ TABLE_1 ,
702
+ generator .generate (
703
+ new Object [] {BinaryStringData .fromString ("class2" ), 2 , 100 }),
704
+ new HashMap <String , String >() {
705
+ {
706
+ put ("timestamp-type" , "type2" );
707
+ }
708
+ });
709
+ split1 .add (insertEvent3 );
710
+
711
+ // add column
712
+ AddColumnEvent .ColumnWithPosition columnWithPosition =
713
+ new AddColumnEvent .ColumnWithPosition (
714
+ Column .physicalColumn ("import-package" , DataTypes .STRING ()));
715
+ AddColumnEvent addColumnEvent =
716
+ new AddColumnEvent (TABLE_1 , Collections .singletonList (columnWithPosition ));
717
+ split1 .add (addColumnEvent );
718
+ schema = SchemaUtils .applySchemaChangeEvent (schema , addColumnEvent );
719
+
720
+ // rename column
721
+ Map <String , String > nameMapping = new HashMap <>();
722
+ nameMapping .put ("bar-foo" , "bar-baz" );
723
+ RenameColumnEvent renameColumnEvent = new RenameColumnEvent (TABLE_1 , nameMapping );
724
+ split1 .add (renameColumnEvent );
725
+ schema = SchemaUtils .applySchemaChangeEvent (schema , renameColumnEvent );
726
+
727
+ // drop column
728
+ DropColumnEvent dropColumnEvent =
729
+ new DropColumnEvent (TABLE_1 , Collections .singletonList ("bar-baz" ));
730
+ split1 .add (dropColumnEvent );
731
+ schema = SchemaUtils .applySchemaChangeEvent (schema , dropColumnEvent );
732
+
733
+ generator = new BinaryRecordDataGenerator ((RowType ) schema .toRowDataType ());
734
+
735
+ // delete
736
+ split1 .add (
737
+ DataChangeEvent .deleteEvent (
738
+ TABLE_1 ,
739
+ generator .generate (
740
+ new Object [] {
741
+ BinaryStringData .fromString ("class1" ),
742
+ 1 ,
743
+ BinaryStringData .fromString ("" ),
744
+ }),
745
+ new HashMap <String , String >() {
746
+ {
747
+ put ("timestamp-type" , "type1" );
748
+ }
749
+ }));
750
+
751
+ // update
752
+ split1 .add (
753
+ DataChangeEvent .updateEvent (
754
+ TABLE_1 ,
755
+ generator .generate (
756
+ new Object [] {
757
+ BinaryStringData .fromString ("class2" ),
758
+ 2 ,
759
+ BinaryStringData .fromString ("" )
760
+ }),
761
+ generator .generate (
762
+ new Object [] {
763
+ BinaryStringData .fromString ("new-class2" ),
764
+ 20 ,
765
+ BinaryStringData .fromString ("new-package2" ),
766
+ }),
767
+ new HashMap <String , String >() {
768
+ {
769
+ put ("timestamp-type" , "type2" );
770
+ }
771
+ }));
772
+
773
+ eventOfSplits .add (split1 );
774
+ return eventOfSplits ;
775
+ }
647
776
}
0 commit comments