1919
2020import com .dtstack .flink .sql .format .FormatType ;
2121import com .dtstack .flink .sql .format .SerializationMetricWrapper ;
22+ import com .dtstack .flink .sql .sink .kafka .serialization .AvroCRowSerializationSchema ;
23+ import com .dtstack .flink .sql .sink .kafka .serialization .CsvCRowSerializationSchema ;
24+ import com .dtstack .flink .sql .sink .kafka .serialization .JsonCRowSerializationSchema ;
2225import com .dtstack .flink .sql .sink .kafka .table .KafkaSinkTableInfo ;
2326import org .apache .commons .lang3 .StringUtils ;
2427import org .apache .flink .api .common .serialization .SerializationSchema ;
2528import org .apache .flink .api .common .typeinfo .TypeInformation ;
2629import org .apache .flink .formats .avro .AvroRowSerializationSchema ;
2730import org .apache .flink .formats .csv .CsvRowSerializationSchema ;
28- import org .apache .flink .formats .json .JsonRowSerializationSchema ;
2931import org .apache .flink .streaming .api .functions .sink .RichSinkFunction ;
3032import org .apache .flink .streaming .connectors .kafka .partitioner .FlinkKafkaPartitioner ;
33+ import org .apache .flink .table .runtime .types .CRow ;
3134import org .apache .flink .types .Row ;
3235
3336import java .util .Optional ;
@@ -51,42 +54,36 @@ public abstract class AbstractKafkaProducerFactory {
5154 * @param partitioner
5255 * @return
5356 */
54- public abstract RichSinkFunction <Row > createKafkaProducer (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <Row > typeInformation , Properties properties , Optional <FlinkKafkaPartitioner <Row >> partitioner , String [] partitionKeys );
57+ public abstract RichSinkFunction <CRow > createKafkaProducer (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <CRow > typeInformation , Properties properties , Optional <FlinkKafkaPartitioner <CRow >> partitioner , String [] partitionKeys );
5558
56- protected SerializationMetricWrapper createSerializationMetricWrapper (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <Row > typeInformation ) {
57- return new SerializationMetricWrapper (createSerializationSchema (kafkaSinkTableInfo , typeInformation ));
59+ protected SerializationMetricWrapper createSerializationMetricWrapper (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <CRow > typeInformation ) {
60+ SerializationSchema <CRow > serializationSchema = createSerializationSchema (kafkaSinkTableInfo , typeInformation );
61+ return new SerializationMetricWrapper (serializationSchema );
5862 }
5963
60- private SerializationSchema <Row > createSerializationSchema (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <Row > typeInformation ) {
61- SerializationSchema <Row > serializationSchema = null ;
64+ private SerializationSchema <CRow > createSerializationSchema (KafkaSinkTableInfo kafkaSinkTableInfo , TypeInformation <CRow > typeInformation ) {
65+ SerializationSchema <CRow > serializationSchema = null ;
6266 if (FormatType .JSON .name ().equalsIgnoreCase (kafkaSinkTableInfo .getSinkDataType ())) {
63-
6467 if (StringUtils .isNotBlank (kafkaSinkTableInfo .getSchemaString ())) {
65- serializationSchema = new JsonRowSerializationSchema (kafkaSinkTableInfo .getSchemaString ());
68+ serializationSchema = new JsonCRowSerializationSchema (kafkaSinkTableInfo .getSchemaString (), kafkaSinkTableInfo . getUpdateMode ());
6669 } else if (typeInformation != null && typeInformation .getArity () != 0 ) {
67- serializationSchema = new JsonRowSerializationSchema (typeInformation );
70+ serializationSchema = new JsonCRowSerializationSchema (typeInformation , kafkaSinkTableInfo . getUpdateMode () );
6871 } else {
6972 throw new IllegalArgumentException ("sinkDataType:" + FormatType .JSON .name () + " must set schemaString(JSON Schema)or TypeInformation<Row>" );
7073 }
71-
7274 } else if (FormatType .CSV .name ().equalsIgnoreCase (kafkaSinkTableInfo .getSinkDataType ())) {
73-
7475 if (StringUtils .isBlank (kafkaSinkTableInfo .getFieldDelimiter ())) {
7576 throw new IllegalArgumentException ("sinkDataType:" + FormatType .CSV .name () + " must set fieldDelimiter" );
7677 }
7778
78- final CsvRowSerializationSchema .Builder serSchemaBuilder = new CsvRowSerializationSchema .Builder (typeInformation );
79+ final CsvCRowSerializationSchema .Builder serSchemaBuilder = new CsvCRowSerializationSchema .Builder (typeInformation );
7980 serSchemaBuilder .setFieldDelimiter (kafkaSinkTableInfo .getFieldDelimiter ().toCharArray ()[0 ]);
8081 serializationSchema = serSchemaBuilder .build ();
81-
8282 } else if (FormatType .AVRO .name ().equalsIgnoreCase (kafkaSinkTableInfo .getSinkDataType ())) {
83-
8483 if (StringUtils .isBlank (kafkaSinkTableInfo .getSchemaString ())) {
8584 throw new IllegalArgumentException ("sinkDataType:" + FormatType .AVRO .name () + " must set schemaString" );
8685 }
87-
88- serializationSchema = new AvroRowSerializationSchema (kafkaSinkTableInfo .getSchemaString ());
89-
86+ serializationSchema = new AvroCRowSerializationSchema (kafkaSinkTableInfo .getSchemaString (),kafkaSinkTableInfo .getUpdateMode ());
9087 }
9188
9289 if (null == serializationSchema ) {
0 commit comments