Skip to content

Commit de8810a

Browse files
committed
use tuple replace crow
1 parent 480cc75 commit de8810a

File tree

19 files changed

+119
-489
lines changed

19 files changed

+119
-489
lines changed

core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@
2121
import com.dtstack.flink.sql.metric.MetricConstant;
2222
import org.apache.flink.api.common.functions.RuntimeContext;
2323
import org.apache.flink.api.common.serialization.SerializationSchema;
24+
import org.apache.flink.api.java.tuple.Tuple2;
2425
import org.apache.flink.metrics.Counter;
2526
import org.apache.flink.metrics.Meter;
2627
import org.apache.flink.metrics.MeterView;
27-
import org.apache.flink.table.runtime.types.CRow;
2828
import org.apache.flink.types.Row;
2929

3030

@@ -35,11 +35,11 @@
3535
* author: toutian
3636
* create: 2019/12/24
3737
*/
38-
public class SerializationMetricWrapper implements SerializationSchema<CRow> {
38+
public class SerializationMetricWrapper implements SerializationSchema<Tuple2<Boolean,Row>> {
3939

4040
private static final long serialVersionUID = 1L;
4141

42-
private SerializationSchema<CRow> serializationSchema;
42+
private SerializationSchema<Tuple2<Boolean,Row>> serializationSchema;
4343

4444
private transient RuntimeContext runtimeContext;
4545

@@ -48,7 +48,7 @@ public class SerializationMetricWrapper implements SerializationSchema<CRow> {
4848
protected transient Meter dtNumRecordsOutRate;
4949

5050

51-
public SerializationMetricWrapper(SerializationSchema<CRow> serializationSchema) {
51+
public SerializationMetricWrapper(SerializationSchema<Tuple2<Boolean,Row>> serializationSchema) {
5252
this.serializationSchema = serializationSchema;
5353
}
5454

@@ -58,7 +58,7 @@ public void initMetric() {
5858
}
5959

6060
@Override
61-
public byte[] serialize(CRow element) {
61+
public byte[] serialize(Tuple2<Boolean,Row> element) {
6262
beforeSerialize();
6363
byte[] row = serializationSchema.serialize(element);
6464
afterSerialize();
@@ -80,7 +80,7 @@ public void setRuntimeContext(RuntimeContext runtimeContext) {
8080
this.runtimeContext = runtimeContext;
8181
}
8282

83-
public SerializationSchema<CRow> getSerializationSchema() {
83+
public SerializationSchema<Tuple2<Boolean,Row>> getSerializationSchema() {
8484
return serializationSchema;
8585
}
8686

kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java

Lines changed: 15 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,16 +19,17 @@
1919

2020
import com.dtstack.flink.sql.format.FormatType;
2121
import 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;
22+
import com.dtstack.flink.sql.sink.kafka.serialization.AvroTuple2SerializationSchema;
23+
import com.dtstack.flink.sql.sink.kafka.serialization.CsvTupleSerializationSchema;
24+
import com.dtstack.flink.sql.sink.kafka.serialization.JsonTupleSerializationSchema;
2525
import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo;
2626
import org.apache.commons.lang3.StringUtils;
2727
import org.apache.flink.api.common.serialization.SerializationSchema;
2828
import org.apache.flink.api.common.typeinfo.TypeInformation;
29+
import org.apache.flink.api.java.tuple.Tuple2;
2930
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
3031
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
31-
import org.apache.flink.table.runtime.types.CRow;
32+
import org.apache.flink.types.Row;
3233

3334
import java.util.Optional;
3435
import java.util.Properties;
@@ -51,28 +52,29 @@ public abstract class AbstractKafkaProducerFactory {
5152
* @param partitioner
5253
* @return
5354
*/
54-
public abstract RichSinkFunction<CRow> createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<CRow> typeInformation, Properties properties, Optional<FlinkKafkaPartitioner<CRow>> partitioner, String[] partitionKeys);
55+
public abstract RichSinkFunction<Tuple2<Boolean,Row>> createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<Tuple2<Boolean,Row>> typeInformation,
56+
Properties properties, Optional<FlinkKafkaPartitioner<Tuple2<Boolean,Row>>> partitioner, String[] partitionKeys);
5557

56-
protected SerializationMetricWrapper createSerializationMetricWrapper(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<CRow> typeInformation) {
57-
SerializationSchema<CRow> serializationSchema = createSerializationSchema(kafkaSinkTableInfo, typeInformation);
58+
protected SerializationMetricWrapper createSerializationMetricWrapper(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<Tuple2<Boolean,Row>> typeInformation) {
59+
SerializationSchema<Tuple2<Boolean,Row>> serializationSchema = createSerializationSchema(kafkaSinkTableInfo, typeInformation);
5860
return new SerializationMetricWrapper(serializationSchema);
5961
}
6062

61-
private SerializationSchema<CRow> createSerializationSchema(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<CRow> typeInformation) {
62-
SerializationSchema<CRow> serializationSchema = null;
63+
private SerializationSchema<Tuple2<Boolean,Row>> createSerializationSchema(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation<Tuple2<Boolean,Row>> typeInformation) {
64+
SerializationSchema<Tuple2<Boolean,Row>> serializationSchema = null;
6365
if (FormatType.JSON.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) {
6466
if (StringUtils.isNotBlank(kafkaSinkTableInfo.getSchemaString())) {
65-
serializationSchema = new JsonCRowSerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode());
67+
serializationSchema = new JsonTupleSerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode());
6668
} else if (typeInformation != null && typeInformation.getArity() != 0) {
67-
serializationSchema = new JsonCRowSerializationSchema(typeInformation, kafkaSinkTableInfo.getUpdateMode());
69+
serializationSchema = new JsonTupleSerializationSchema(typeInformation, kafkaSinkTableInfo.getUpdateMode());
6870
} else {
6971
throw new IllegalArgumentException("sinkDataType:" + FormatType.JSON.name() + " must set schemaString(JSON Schema)or TypeInformation<Row>");
7072
}
7173
} else if (FormatType.CSV.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) {
7274
if (StringUtils.isBlank(kafkaSinkTableInfo.getFieldDelimiter())) {
7375
throw new IllegalArgumentException("sinkDataType:" + FormatType.CSV.name() + " must set fieldDelimiter");
7476
}
75-
final CsvCRowSerializationSchema.Builder serSchemaBuilder = new CsvCRowSerializationSchema.Builder(typeInformation);
77+
final CsvTupleSerializationSchema.Builder serSchemaBuilder = new CsvTupleSerializationSchema.Builder(typeInformation);
7678
serSchemaBuilder.setFieldDelimiter(kafkaSinkTableInfo.getFieldDelimiter().toCharArray()[0]);
7779
serSchemaBuilder.setUpdateMode(kafkaSinkTableInfo.getUpdateMode());
7880

@@ -81,7 +83,7 @@ private SerializationSchema<CRow> createSerializationSchema(KafkaSinkTableInfo k
8183
if (StringUtils.isBlank(kafkaSinkTableInfo.getSchemaString())) {
8284
throw new IllegalArgumentException("sinkDataType:" + FormatType.AVRO.name() + " must set schemaString");
8385
}
84-
serializationSchema = new AvroCRowSerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode());
86+
serializationSchema = new AvroTuple2SerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode());
8587
}
8688

8789
if (null == serializationSchema) {

0 commit comments

Comments
 (0)