forked from DTStack/flinkStreamSQL
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge remote-tracking branch 'origin/feat_1.8_codereview' into 1.8_te…
…st_3.10.x
- Loading branch information
Showing
5 changed files
with
186 additions
and
455 deletions.
There are no files selected for viewing
148 changes: 148 additions & 0 deletions
148
...ase/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,148 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package com.dtstack.flink.sql.sink.kafka; | ||
|
||
import com.dtstack.flink.sql.sink.IStreamSinkGener; | ||
import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; | ||
import org.apache.commons.lang3.StringUtils; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.api.java.tuple.Tuple2; | ||
import org.apache.flink.api.java.typeutils.RowTypeInfo; | ||
import org.apache.flink.api.java.typeutils.TupleTypeInfo; | ||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
import org.apache.flink.streaming.api.functions.sink.SinkFunction; | ||
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; | ||
import org.apache.flink.table.api.TableSchema; | ||
import org.apache.flink.table.runtime.types.CRow; | ||
import org.apache.flink.table.runtime.types.CRowTypeInfo; | ||
import org.apache.flink.table.sinks.RetractStreamTableSink; | ||
import org.apache.flink.table.sinks.TableSink; | ||
import org.apache.flink.types.Row; | ||
import org.apache.flink.util.Preconditions; | ||
import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
|
||
import java.util.Optional; | ||
import java.util.Properties; | ||
import java.util.stream.IntStream; | ||
|
||
/** | ||
* Date: 2020/4/1 | ||
* Company: www.dtstack.com | ||
* @author maqi | ||
*/ | ||
public abstract class AbstractKafkaSink implements RetractStreamTableSink<Row>, IStreamSinkGener { | ||
|
||
public static final String SINK_OPERATOR_NAME_TPL = "${topic}_${table}"; | ||
|
||
protected String[] fieldNames; | ||
protected TypeInformation<?>[] fieldTypes; | ||
|
||
protected String[] partitionKeys; | ||
protected String sinkOperatorName; | ||
protected Properties properties; | ||
protected int parallelism; | ||
protected String topic; | ||
protected String tableName; | ||
|
||
protected TableSchema schema; | ||
protected SinkFunction<CRow> kafkaProducer; | ||
|
||
|
||
protected Optional<FlinkKafkaPartitioner<CRow>> partitioner; | ||
|
||
protected Properties getKafkaProperties(KafkaSinkTableInfo KafkaSinkTableInfo) { | ||
Properties props = new Properties(); | ||
props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaSinkTableInfo.getBootstrapServers()); | ||
|
||
for (String key : KafkaSinkTableInfo.getKafkaParamKeys()) { | ||
props.setProperty(key, KafkaSinkTableInfo.getKafkaParam(key)); | ||
} | ||
return props; | ||
} | ||
|
||
protected TypeInformation[] getTypeInformations(KafkaSinkTableInfo kafka11SinkTableInfo) { | ||
Class<?>[] fieldClasses = kafka11SinkTableInfo.getFieldClasses(); | ||
TypeInformation[] types = IntStream.range(0, fieldClasses.length) | ||
.mapToObj(i -> TypeInformation.of(fieldClasses[i])) | ||
.toArray(TypeInformation[]::new); | ||
return types; | ||
} | ||
|
||
|
||
protected TableSchema buildTableSchema(String[] fieldNames, TypeInformation<?>[] fieldTypes) { | ||
Preconditions.checkArgument(fieldNames.length == fieldTypes.length, "fieldNames length must equals fieldTypes length !"); | ||
|
||
TableSchema.Builder builder = TableSchema.builder(); | ||
IntStream.range(0, fieldTypes.length) | ||
.forEach(i -> builder.field(fieldNames[i], fieldTypes[i])); | ||
|
||
return builder.build(); | ||
} | ||
|
||
|
||
@Override | ||
public void emitDataStream(DataStream<Tuple2<Boolean, Row>> dataStream) { | ||
DataStream<CRow> mapDataStream = dataStream | ||
.map((Tuple2<Boolean, Row> record) -> new CRow(record.f1, record.f0)) | ||
.returns(getRowTypeInfo()) | ||
.setParallelism(parallelism); | ||
|
||
mapDataStream.addSink(kafkaProducer).name(sinkOperatorName); | ||
} | ||
|
||
public CRowTypeInfo getRowTypeInfo() { | ||
return new CRowTypeInfo(new RowTypeInfo(fieldTypes, fieldNames)); | ||
} | ||
|
||
protected String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo) { | ||
if (StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())) { | ||
return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); | ||
} | ||
return null; | ||
} | ||
|
||
@Override | ||
public TupleTypeInfo<Tuple2<Boolean, Row>> getOutputType() { | ||
return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); | ||
} | ||
|
||
@Override | ||
public String[] getFieldNames() { | ||
return fieldNames; | ||
} | ||
|
||
@Override | ||
public TypeInformation<?>[] getFieldTypes() { | ||
return fieldTypes; | ||
} | ||
|
||
@Override | ||
public TableSink<Tuple2<Boolean, Row>> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) { | ||
this.fieldNames = fieldNames; | ||
this.fieldTypes = fieldTypes; | ||
return this; | ||
} | ||
|
||
@Override | ||
public TypeInformation<Row> getRecordType() { | ||
return new RowTypeInfo(fieldTypes, fieldNames); | ||
} | ||
|
||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.