2828import org .apache .commons .collections .CollectionUtils ;
2929import org .apache .commons .lang3 .StringUtils ;
3030import org .apache .flink .streaming .api .datastream .DataStream ;
31+ import org .apache .flink .table .api .Schema ;
32+ import org .apache .flink .table .catalog .Column ;
33+ import org .apache .flink .table .catalog .ResolvedSchema ;
34+ import org .apache .flink .table .catalog .UniqueConstraint ;
3135import org .apache .flink .table .operations .Operation ;
36+ import org .apache .flink .table .types .AtomicDataType ;
3237import org .apache .flink .types .Row ;
3338
3439import java .io .Serializable ;
40+ import java .util .ArrayList ;
3541import java .util .List ;
3642
3743public class SQLSinkBuilder extends AbstractSqlSinkBuilder implements Serializable {
@@ -48,9 +54,19 @@ private SQLSinkBuilder(FlinkCDCConfig config) {
4854 private FlinkTableObjectIdentifier addSourceTableView (DataStream <Row > rowDataDataStream , Table table ) {
4955 // Because the name of the view on Flink is not allowed to have -, it needs to be replaced with - here_
5056 String viewName = replaceViewNameMiddleLineToUnderLine ("VIEW_" + table .getSchemaTableNameWithUnderline ());
51-
57+ final ResolvedSchema resolvedSchema =
58+ customTableEnvironment .fromChangelogStream (rowDataDataStream ).getResolvedSchema ();
59+ List <Column > columns = new ArrayList <>();
60+ for (Column column : resolvedSchema .getColumns ()) {
61+ columns .add (column .copy (new AtomicDataType (
62+ column .getDataType ().getLogicalType ().copy (false ),
63+ column .getDataType ().getConversionClass ())));
64+ }
65+ final UniqueConstraint primaryKey = UniqueConstraint .primaryKey (viewName + "_pk" , table .getPrimaryKeys ());
66+ final ResolvedSchema sinkSchema = new ResolvedSchema (columns , resolvedSchema .getWatermarkSpecs (), primaryKey );
67+ final Schema schema = Schema .newBuilder ().fromResolvedSchema (sinkSchema ).build ();
5268 customTableEnvironment .createTemporaryView (
53- viewName , customTableEnvironment .fromChangelogStream (rowDataDataStream ));
69+ viewName , customTableEnvironment .fromChangelogStream (rowDataDataStream , schema ));
5470 logger .info ("Create {} temporaryView successful..." , viewName );
5571 return FlinkTableObjectIdentifier .of (viewName );
5672 }
0 commit comments