Skip to content

Commit 652f6be

Browse files
committed
Merge remote-tracking branch 'origin/v1.5.0_dev' into 1.8dev_merge1.5
# Conflicts: # kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java # kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java # kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java
2 parents 4a87626 + 86bd0ec commit 652f6be

File tree

3 files changed

+92
-142
lines changed
  • kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka
  • kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka
  • kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka

3 files changed

+92
-142
lines changed

kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java

Lines changed: 27 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -23,17 +23,14 @@
2323
import com.dtstack.flink.sql.table.TargetTableInfo;
2424
import org.apache.flink.api.common.serialization.SerializationSchema;
2525
import org.apache.flink.api.common.typeinfo.TypeInformation;
26+
import org.apache.flink.api.java.tuple.Tuple2;
2627
import org.apache.flink.api.java.typeutils.RowTypeInfo;
28+
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
2729
import org.apache.flink.streaming.api.datastream.DataStream;
28-
import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase;
29-
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
30-
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
31-
import org.apache.flink.table.api.TableSchema;
32-
import org.apache.flink.table.sinks.AppendStreamTableSink;
30+
import org.apache.flink.streaming.connectors.kafka.KafkaTableSink;
31+
import org.apache.flink.table.sinks.RetractStreamTableSink;
3332
import org.apache.flink.table.sinks.TableSink;
3433
import org.apache.flink.types.Row;
35-
36-
import java.util.Optional;
3734
import java.util.Properties;
3835

3936
/**
@@ -43,78 +40,64 @@
4340
* @author DocLi
4441
* @modifyer maqi
4542
*/
46-
public class KafkaSink implements AppendStreamTableSink<Row>, IStreamSinkGener<KafkaSink> {
43+
public class KafkaSink implements RetractStreamTableSink<Row>, IStreamSinkGener<KafkaSink> {
4744

4845
protected String[] fieldNames;
4946

5047
protected TypeInformation<?>[] fieldTypes;
5148

52-
/** The schema of the table. */
53-
private TableSchema schema;
54-
55-
/** The Kafka topic to write to. */
5649
protected String topic;
5750

58-
/** Properties for the Kafka producer. */
5951
protected Properties properties;
6052

6153
/** Serialization schema for encoding records to Kafka. */
6254
protected SerializationSchema serializationSchema;
6355

64-
/** Partitioner to select Kafka partition for each item. */
65-
protected Optional<FlinkKafkaPartitioner<Row>> partitioner;
66-
6756
@Override
6857
public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) {
69-
7058
KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo;
7159
this.topic = kafka09SinkTableInfo.getTopic();
72-
73-
Properties props = new Properties();
74-
props.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers());
75-
76-
for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) {
77-
props.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key));
78-
}
79-
this.properties = props;
80-
this.partitioner = Optional.of(new FlinkFixedPartitioner<>());
8160
this.fieldNames = kafka09SinkTableInfo.getFields();
8261
TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length];
83-
for(int i = 0; i< kafka09SinkTableInfo.getFieldClasses().length; i++){
62+
for (int i = 0; i < kafka09SinkTableInfo.getFieldClasses().length; i++) {
8463
types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]);
8564
}
8665
this.fieldTypes = types;
8766

88-
TableSchema.Builder schemaBuilder = TableSchema.builder();
89-
for (int i=0;i<fieldNames.length;i++){
90-
schemaBuilder.field(fieldNames[i], fieldTypes[i]);
91-
}
92-
this.schema = schemaBuilder.build();
93-
94-
//this.serializationSchema = Optional.of(JsonRowSerializationSchema.class);
95-
if ("json".equalsIgnoreCase(kafka09SinkTableInfo.getSinkDataType())) {
96-
this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType());
67+
properties = new Properties();
68+
for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) {
69+
properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key));
9770
}
71+
properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers());
9872

73+
this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType().getTypeAt(1));
9974
return this;
10075
}
10176

10277
@Override
103-
public void emitDataStream(DataStream<Row> dataStream) {
104-
KafkaTableSinkBase kafkaTableSink = new CustomerKafka09JsonTableSink(
105-
schema,
78+
public TypeInformation<Row> getRecordType() {
79+
return new RowTypeInfo(fieldTypes, fieldNames);
80+
}
81+
82+
@Override
83+
public void emitDataStream(DataStream<Tuple2<Boolean, Row>> dataStream) {
84+
KafkaTableSink kafkaTableSink = new CustomerKafka09JsonTableSink(
10685
topic,
10786
properties,
108-
partitioner,
10987
serializationSchema
11088
);
11189

112-
kafkaTableSink.emitDataStream(dataStream);
90+
91+
DataStream<Row> ds = dataStream.map((Tuple2<Boolean, Row> record) -> {
92+
return record.f1;
93+
}).returns(getOutputType().getTypeAt(1));
94+
95+
kafkaTableSink.emitDataStream(ds);
11396
}
11497

11598
@Override
116-
public TypeInformation<Row> getOutputType() {
117-
return new RowTypeInfo(fieldTypes, fieldNames);
99+
public TupleTypeInfo<Tuple2<Boolean, Row>> getOutputType() {
100+
return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames));
118101
}
119102

120103
@Override
@@ -128,7 +111,7 @@ public TypeInformation<?>[] getFieldTypes() {
128111
}
129112

130113
@Override
131-
public TableSink<Row> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
114+
public TableSink<Tuple2<Boolean, Row>> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
132115
this.fieldNames = fieldNames;
133116
this.fieldTypes = fieldTypes;
134117
return this;

kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java

Lines changed: 32 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -23,13 +23,12 @@
2323
import com.dtstack.flink.sql.table.TargetTableInfo;
2424
import org.apache.flink.api.common.serialization.SerializationSchema;
2525
import org.apache.flink.api.common.typeinfo.TypeInformation;
26+
import org.apache.flink.api.java.tuple.Tuple2;
2627
import org.apache.flink.api.java.typeutils.RowTypeInfo;
28+
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
2729
import org.apache.flink.streaming.api.datastream.DataStream;
28-
import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase;
29-
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
30-
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
31-
import org.apache.flink.table.api.TableSchema;
32-
import org.apache.flink.table.sinks.AppendStreamTableSink;
30+
import org.apache.flink.streaming.connectors.kafka.KafkaTableSink;
31+
import org.apache.flink.table.sinks.RetractStreamTableSink;
3332
import org.apache.flink.table.sinks.TableSink;
3433
import org.apache.flink.types.Row;
3534

@@ -45,76 +44,65 @@
4544
* @modifyer maqi
4645
*
4746
*/
48-
public class KafkaSink implements AppendStreamTableSink<Row>, IStreamSinkGener<KafkaSink> {
47+
public class KafkaSink implements RetractStreamTableSink<Row>, IStreamSinkGener<KafkaSink> {
4948

5049

5150
protected String[] fieldNames;
5251

5352
protected TypeInformation<?>[] fieldTypes;
5453

55-
/** The schema of the table. */
56-
private TableSchema schema;
57-
58-
/** The Kafka topic to write to. */
5954
protected String topic;
6055

61-
/** Properties for the Kafka producer. */
6256
protected Properties properties;
6357

6458
/** Serialization schema for encoding records to Kafka. */
6559
protected SerializationSchema serializationSchema;
6660

67-
/** Partitioner to select Kafka partition for each item. */
68-
protected Optional<FlinkKafkaPartitioner<Row>> partitioner;
6961
@Override
7062
public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) {
71-
KafkaSinkTableInfo kafka010SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo;
72-
this.topic = kafka010SinkTableInfo.getTopic();
73-
74-
Properties props = new Properties();
75-
props.setProperty("bootstrap.servers", kafka010SinkTableInfo.getBootstrapServers());
76-
77-
for (String key:kafka010SinkTableInfo.getKafkaParamKeys()) {
78-
props.setProperty(key, kafka010SinkTableInfo.getKafkaParam(key));
79-
}
80-
this.properties = props;
81-
this.partitioner = Optional.of(new FlinkFixedPartitioner<>());
82-
this.fieldNames = kafka010SinkTableInfo.getFields();
83-
TypeInformation[] types = new TypeInformation[kafka010SinkTableInfo.getFields().length];
84-
for(int i = 0; i< kafka010SinkTableInfo.getFieldClasses().length; i++){
85-
types[i] = TypeInformation.of(kafka010SinkTableInfo.getFieldClasses()[i]);
63+
KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo;
64+
this.topic = kafka10SinkTableInfo.getTopic();
65+
this.fieldNames = kafka10SinkTableInfo.getFields();
66+
TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length];
67+
for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) {
68+
types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]);
8669
}
8770
this.fieldTypes = types;
8871

89-
TableSchema.Builder schemaBuilder = TableSchema.builder();
90-
for (int i=0;i<fieldNames.length;i++){
91-
schemaBuilder.field(fieldNames[i], fieldTypes[i]);
72+
properties = new Properties();
73+
for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) {
74+
properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key));
9275
}
93-
this.schema = schemaBuilder.build();
76+
properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers());
9477

95-
//this.serializationSchema = Optional.of(JsonRowSerializationSchema.class);
96-
if ("json".equalsIgnoreCase(kafka010SinkTableInfo.getSinkDataType())) {
97-
this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType());
98-
}
78+
this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType().getTypeAt(1));
9979
return this;
10080
}
10181

10282
@Override
103-
public void emitDataStream(DataStream<Row> dataStream) {
104-
KafkaTableSinkBase kafkaTableSink = new CustomerKafka10JsonTableSink(
105-
schema,
83+
public TypeInformation<Row> getRecordType() {
84+
return new RowTypeInfo(fieldTypes, fieldNames);
85+
}
86+
87+
@Override
88+
public void emitDataStream(DataStream<Tuple2<Boolean, Row>> dataStream) {
89+
KafkaTableSink kafkaTableSink = new CustomerKafka10JsonTableSink(
10690
topic,
10791
properties,
108-
partitioner,
10992
serializationSchema
11093
);
11194

112-
kafkaTableSink.emitDataStream(dataStream);
95+
96+
DataStream<Row> ds = dataStream.map((Tuple2<Boolean, Row> record) -> {
97+
return record.f1;
98+
}).returns(getOutputType().getTypeAt(1));
99+
100+
kafkaTableSink.emitDataStream(ds);
113101
}
114102

115103
@Override
116-
public TypeInformation<Row> getOutputType() {
117-
return new RowTypeInfo(fieldTypes, fieldNames);
104+
public TupleTypeInfo<Tuple2<Boolean, Row>> getOutputType() {
105+
return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames));
118106
}
119107

120108
@Override
@@ -128,7 +116,7 @@ public TypeInformation<?>[] getFieldTypes() {
128116
}
129117

130118
@Override
131-
public TableSink<Row> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
119+
public TableSink<Tuple2<Boolean, Row>> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
132120
this.fieldNames = fieldNames;
133121
this.fieldTypes = fieldTypes;
134122
return this;

0 commit comments

Comments
 (0)