-
Notifications
You must be signed in to change notification settings - Fork 386
/
TransactionalProducerStage.scala
282 lines (240 loc) · 10.5 KB
/
TransactionalProducerStage.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
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
/*
* Copyright (C) 2014 - 2016 Softwaremill <https://softwaremill.com>
* Copyright (C) 2016 - 2020 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.kafka.internal
import akka.Done
import akka.annotation.InternalApi
import akka.kafka.ConsumerMessage.{GroupTopicPartition, PartitionOffsetCommittedMarker}
import akka.kafka.ProducerMessage.{Envelope, Results}
import akka.kafka.internal.DeferredProducer._
import akka.kafka.internal.ProducerStage.ProducerCompletionState
import akka.kafka.{ConsumerMessage, ProducerSettings}
import akka.stream.stage._
import akka.stream.{Attributes, FlowShape}
import org.apache.kafka.clients.consumer.{ConsumerGroupMetadata, OffsetAndMetadata}
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.common.TopicPartition
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.jdk.CollectionConverters._
/**
* INTERNAL API
*/
@InternalApi
private[kafka] final class TransactionalProducerStage[K, V, P](
val settings: ProducerSettings[K, V],
transactionalId: String
) extends GraphStage[FlowShape[Envelope[K, V, P], Future[Results[K, V, P]]]]
with ProducerStage[K, V, P, Envelope[K, V, P], Results[K, V, P]] {
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TransactionalProducerStageLogic(this, transactionalId, inheritedAttributes)
}
/** Internal API */
private object TransactionalProducerStage {
object TransactionBatch {
def empty: TransactionBatch = new EmptyTransactionBatch()
}
private[kafka] sealed trait TransactionBatch {
def updated(partitionOffset: PartitionOffsetCommittedMarker): TransactionBatch
def committingFailed(): Unit
}
final class EmptyTransactionBatch extends TransactionBatch {
override def updated(partitionOffset: PartitionOffsetCommittedMarker): TransactionBatch =
new NonemptyTransactionBatch(partitionOffset)
override def committingFailed(): Unit = {}
}
final class NonemptyTransactionBatch(head: PartitionOffsetCommittedMarker,
tail: Map[GroupTopicPartition, Long] = Map[GroupTopicPartition, Long]())
extends TransactionBatch {
// There is no guarantee that offsets adding callbacks will be called in any particular order.
// Decreasing an offset stored for the KTP would mean possible data duplication.
// Since `awaitingConfirmation` counter guarantees that all writes finished, we can safely assume
// that all all data up to maximal offsets has been wrote to Kafka.
private val previousHighest = tail.getOrElse(head.key, -1L)
private[internal] val offsets = tail + (head.key -> head.offset.max(previousHighest))
def group: String = head.key.groupId
def committedMarker: CommittedMarker = head.committedMarker
def offsetMap(): Map[TopicPartition, OffsetAndMetadata] = offsets.map {
case (gtp, offset) => new TopicPartition(gtp.topic, gtp.partition) -> new OffsetAndMetadata(offset + 1)
}
def internalCommit(): Future[Done] =
committedMarker.committed(offsetMap())
override def committingFailed(): Unit =
committedMarker.failed()
override def updated(partitionOffset: PartitionOffsetCommittedMarker): TransactionBatch = {
require(
group == partitionOffset.key.groupId,
s"Transaction batch must contain messages from exactly 1 consumer group. $group != ${partitionOffset.key.groupId}"
)
require(
this.committedMarker == partitionOffset.committedMarker,
"Transaction batch must contain messages from a single source"
)
new NonemptyTransactionBatch(partitionOffset, offsets)
}
}
}
/**
* Internal API.
*
* Transaction (Exactly-Once) Producer State Logic
*/
private final class TransactionalProducerStageLogic[K, V, P](
stage: TransactionalProducerStage[K, V, P],
transactionalId: String,
inheritedAttributes: Attributes
) extends DefaultProducerStageLogic[K, V, P, Envelope[K, V, P], Results[K, V, P]](stage, inheritedAttributes)
with StageIdLogging
with ProducerCompletionState {
import TransactionalProducerStage._
private val commitSchedulerKey = "commit"
private val messageDrainInterval = 10.milliseconds
private var batchOffsets = TransactionBatch.empty
private var demandSuspended = false
private var firstMessage: Option[Envelope[K, V, P]] = None
override protected def logSource: Class[_] = classOf[TransactionalProducerStage[_, _, _]]
// we need to peek at the first message to generate the producer transactional id for partitioned sources
override def preStart(): Unit = resumeDemand()
override protected def producerAssigned(): Unit = {
producingInHandler()
initTransactions()
beginTransaction()
produceFirstMessage()
resumeDemand()
scheduleOnce(commitSchedulerKey, producerSettings.eosCommitInterval)
}
private def produceFirstMessage(): Unit = firstMessage match {
case Some(msg) =>
produce(msg)
firstMessage = None
case _ =>
throw new IllegalStateException("Should never attempt to produce first message if it does not exist.")
}
override protected def resumeDemand(tryToPull: Boolean = true): Unit = {
super.resumeDemand(tryToPull)
demandSuspended = false
}
override protected def suspendDemand(): Unit = {
if (!demandSuspended) super.suspendDemand()
demandSuspended = true
}
override protected def initialInHandler(): Unit =
setHandler(stage.in, new DefaultInHandler {
override def onPush(): Unit = parseFirstMessage(grab(stage.in))
})
override protected def onTimer(timerKey: Any): Unit =
if (timerKey == commitSchedulerKey) {
maybeCommitTransaction()
}
private def maybeCommitTransaction(beginNewTransaction: Boolean = true,
abortEmptyTransactionOnComplete: Boolean = false): Unit = {
val awaitingConf = awaitingConfirmationValue
batchOffsets match {
case batch: NonemptyTransactionBatch if awaitingConf == 0 =>
commitTransaction(batch, beginNewTransaction)
case _: EmptyTransactionBatch if awaitingConf == 0 && abortEmptyTransactionOnComplete =>
abortTransaction("Transaction is empty and stage is completing")
case _ if awaitingConf > 0 =>
suspendDemand()
scheduleOnce(commitSchedulerKey, messageDrainInterval)
case _ =>
scheduleOnce(commitSchedulerKey, producerSettings.eosCommitInterval)
}
}
/**
* When using partitioned sources we extract the transactional id, group id, and topic partition information from
* the first message in order to define a `transacitonal.id` before constructing the [[org.apache.kafka.clients.producer.KafkaProducer]]
*/
private def parseFirstMessage(msg: Envelope[K, V, P]): Boolean =
producerAssignmentLifecycle match {
case Assigned => true
case Unassigned if firstMessage.nonEmpty =>
// this should never happen because demand should be suspended until the producer is assigned
throw new IllegalStateException("Cannot reapply first message")
case Unassigned =>
// stash the first message so it can be sent after the producer is assigned
firstMessage = Some(msg)
// initiate async async producer request _after_ first message is stashed in case future eagerly resolves
// instead of asynccallback
resolveProducer(generatedTransactionalConfig(msg))
// suspend demand after we receive the first message until the producer is assigned
suspendDemand()
false
case AsyncCreateRequestSent =>
throw new IllegalStateException(
s"Should never receive new messages while in producer assignment state '$AsyncCreateRequestSent'"
)
}
private def generatedTransactionalConfig(msg: Envelope[K, V, P]): ProducerSettings[K, V] = {
val txId = msg.passThrough match {
case committedMarker: PartitionOffsetCommittedMarker if committedMarker.fromPartitionedSource =>
val gtp = committedMarker.key
val txId = s"$transactionalId-${gtp.groupId}-${gtp.topic}-${gtp.partition}"
log.debug("Generated transactional id from partitioned source '{}'", txId)
txId
case _ => transactionalId
}
stage.settings.withProperties(
ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG -> true.toString,
ProducerConfig.TRANSACTIONAL_ID_CONFIG -> txId,
ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION -> 1.toString
)
}
override protected def postSend(msg: Envelope[K, V, P]): Unit = msg.passThrough match {
case o: ConsumerMessage.PartitionOffsetCommittedMarker => batchOffsets = batchOffsets.updated(o)
case _ =>
}
override def onCompletionSuccess(): Unit = {
log.debug("Committing final transaction before shutdown")
cancelTimer(commitSchedulerKey)
maybeCommitTransaction(beginNewTransaction = false, abortEmptyTransactionOnComplete = true)
super.onCompletionSuccess()
}
override def onCompletionFailure(ex: Throwable): Unit = {
abortTransaction("Stage failure")
batchOffsets.committingFailed()
super.onCompletionFailure(ex)
}
private def commitTransaction(batch: NonemptyTransactionBatch, beginNewTransaction: Boolean): Unit = {
val group = batch.group
log.debug("Committing transaction for transactional id '{}' consumer group '{}' with offsets: {}",
transactionalId,
group,
batch.offsets)
val offsetMap = batch.offsetMap()
producer.sendOffsetsToTransaction(offsetMap.asJava, new ConsumerGroupMetadata(group))
producer.commitTransaction()
log.debug("Committed transaction for transactional id '{}' consumer group '{}' with offsets: {}",
transactionalId,
group,
batch.offsets)
batchOffsets = TransactionBatch.empty
batch
.internalCommit()
.onComplete { _ =>
onInternalCommitAckCb.invoke(())
}(materializer.executionContext)
if (beginNewTransaction) {
beginTransaction()
resumeDemand()
}
}
private val onInternalCommitAckCb: AsyncCallback[Unit] = {
getAsyncCallback[Unit](
_ => scheduleOnce(commitSchedulerKey, producerSettings.eosCommitInterval)
)
}
private def initTransactions(): Unit = {
log.debug("Initializing transactions")
producer.initTransactions()
}
private def beginTransaction(): Unit = {
log.debug("Beginning new transaction")
producer.beginTransaction()
}
private def abortTransaction(reason: String): Unit = {
log.debug("Aborting transaction: {}", reason)
if (producerAssignmentLifecycle == Assigned) producer.abortTransaction()
}
}