-
Notifications
You must be signed in to change notification settings - Fork 4
/
KafkaSinkConfig.scala
158 lines (138 loc) · 4.75 KB
/
KafkaSinkConfig.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
package io.epiphanous.flinkrunner.model.sink
import com.typesafe.scalalogging.LazyLogging
import io.epiphanous.flinkrunner.model._
import io.epiphanous.flinkrunner.serde.{
ConfluentAvroRegistryKafkaRecordSerializationSchema,
JsonKafkaRecordSerializationSchema
}
import io.epiphanous.flinkrunner.util.ConfigToProps
import io.epiphanous.flinkrunner.util.ConfigToProps.getFromEither
import org.apache.avro.generic.GenericRecord
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.connector.base.DeliveryGuarantee
import org.apache.flink.connector.kafka.sink.{
KafkaRecordSerializationSchema,
KafkaSink
}
import org.apache.flink.streaming.api.datastream.DataStreamSink
import org.apache.flink.streaming.api.scala.DataStream
import java.time.Duration
import java.util.Properties
/** Kafka sink config.
*
* Configuration:
*
* @param name
* name of the sink
* @param config
* flinkrunner config
* @tparam ADT
* the flinkrunner algebraic data type
*/
case class KafkaSinkConfig[ADT <: FlinkEvent: TypeInformation](
name: String,
config: FlinkConfig
) extends SinkConfig[ADT]
with LazyLogging {
override val connector: FlinkConnectorName = FlinkConnectorName.Kafka
override val properties: Properties = ConfigToProps.normalizeProps(
config,
pfx(),
List("bootstrap.servers")
)
val bootstrapServers: String =
properties.getProperty("bootstrap.servers")
val topic: String = config.getString(pfx("topic"))
val isKeyed: Boolean = getFromEither(
pfx(),
Seq("keyed", "is.keyed"),
config.getBooleanOpt
).getOrElse(false)
def deliveryGuarantee: DeliveryGuarantee = config
.getStringOpt(pfx("delivery.guarantee"))
.map(s => s.toLowerCase.replaceAll("[^a-z]+", "-")) match {
case Some("exactly-once") =>
DeliveryGuarantee.EXACTLY_ONCE
case Some("none") =>
DeliveryGuarantee.NONE
case _ => DeliveryGuarantee.AT_LEAST_ONCE
}
/** ensure transaction.timeout.ms is set */
val transactionTimeoutMs: Long = {
val tms = getFromEither(
pfx(),
Seq("transaction.timeout.ms", "tx.timeout.ms"),
config.getLongOpt
)
val td = getFromEither(
pfx(),
Seq("transaction.timeout", "tx.timeout"),
config.getDurationOpt
)
val t = tms.getOrElse(td.getOrElse(Duration.ofHours(2)).toMillis)
properties.setProperty("transaction.timeout.ms", t.toString)
t
}
val transactionalIdPrefix: String =
getFromEither(
pfx(),
Seq(
"transactional.id.prefix",
"transactional.prefix",
"transactional.id",
"transaction.id.prefix",
"transaction.prefix",
"transaction.id",
"tx.id.prefix",
"tx.prefix",
"tx.id"
),
config.getStringOpt
).getOrElse(
s"${config.jobName}.$name.tx.id"
)
val schemaRegistryConfig: SchemaRegistryConfig = SchemaRegistryConfig(
isDeserializing = false,
config
.getObjectOption(pfx("schema.registry"))
)
val cacheConcurrencyLevel: Int =
config.getIntOpt(pfx("cache.concurrency.level")).getOrElse(4)
val cacheMaxSize: Long =
config.getLongOpt(pfx("cache.max.size")).getOrElse(10000L)
val cacheExpireAfter: Duration = config
.getDurationOpt(pfx("cache.expire.after"))
.getOrElse(Duration.ofHours(1))
val cacheRecordStats: Boolean =
config.getBooleanOpt(pfx("cache.record.stats")).getOrElse(true)
/** Return an confluent avro serialization schema */
def getAvroSerializationSchema[
E <: ADT with EmbeddedAvroRecord[A],
A <: GenericRecord]: KafkaRecordSerializationSchema[E] = {
new ConfluentAvroRegistryKafkaRecordSerializationSchema[E, A, ADT](
this
)
}
/** Returns, by default, a json serialization schema */
def getSerializationSchema[E <: ADT: TypeInformation]
: KafkaRecordSerializationSchema[E] =
new JsonKafkaRecordSerializationSchema[E, ADT](this)
override def getAvroSink[
E <: ADT with EmbeddedAvroRecord[A]: TypeInformation,
A <: GenericRecord: TypeInformation](
dataStream: DataStream[E]): DataStreamSink[E] =
dataStream.sinkTo(_getSink[E](getAvroSerializationSchema[E, A])).setParallelism(parallelism)
override def getSink[E <: ADT: TypeInformation](
dataStream: DataStream[E]): DataStreamSink[E] =
dataStream.sinkTo(_getSink[E](getSerializationSchema[E])).setParallelism(parallelism)
def _getSink[E <: ADT: TypeInformation](
serializer: KafkaRecordSerializationSchema[E]): KafkaSink[E] =
KafkaSink
.builder()
.setBootstrapServers(bootstrapServers)
.setDeliveryGuarantee(deliveryGuarantee)
.setTransactionalIdPrefix(transactionalIdPrefix)
.setKafkaProducerConfig(properties)
.setRecordSerializer(serializer)
.build()
}