-
Notifications
You must be signed in to change notification settings - Fork 386
/
BaseSingleSourceLogic.scala
124 lines (106 loc) · 4.15 KB
/
BaseSingleSourceLogic.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
/*
* Copyright (C) 2014 - 2016 Softwaremill <https://softwaremill.com>
* Copyright (C) 2016 - 2023 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.kafka.internal
import akka.actor.{ActorRef, Status, Terminated}
import akka.annotation.InternalApi
import akka.kafka.Subscriptions.{Assignment, AssignmentOffsetsForTimes, AssignmentWithOffset}
import akka.kafka.{ConsumerFailed, ManualSubscription}
import akka.stream.SourceShape
import akka.stream.stage.GraphStageLogic.StageActor
import akka.stream.stage.{AsyncCallback, GraphStageLogic, OutHandler}
import org.apache.kafka.common.TopicPartition
import scala.annotation.tailrec
import scala.concurrent.{ExecutionContext, Future}
/**
* Internal API.
*
* Shared GraphStageLogic for [[SingleSourceLogic]] and [[ExternalSingleSourceLogic]].
*/
@InternalApi private abstract class BaseSingleSourceLogic[K, V, Msg](
val shape: SourceShape[Msg]
) extends GraphStageLogic(shape)
with PromiseControl
with MetricsControl
with StageIdLogging
with SourceLogicSubscription
with MessageBuilder[K, V, Msg]
with SourceLogicBuffer[K, V, Msg] {
override protected def executionContext: ExecutionContext = materializer.executionContext
protected def consumerFuture: Future[ActorRef]
protected final var consumerActor: ActorRef = _
protected var sourceActor: StageActor = _
protected var tps = Set.empty[TopicPartition]
private var requested = false
private var requestId = 0
private val assignedCB: AsyncCallback[Set[TopicPartition]] = getAsyncCallback[Set[TopicPartition]] { assignedTps =>
tps ++= assignedTps
log.debug("Assigned partitions: {}. All partitions: {}", assignedTps, tps)
requestMessages()
}
private val revokedCB: AsyncCallback[Set[TopicPartition]] = getAsyncCallback[Set[TopicPartition]] { revokedTps =>
tps --= revokedTps
log.debug("Revoked partitions: {}. All partitions: {}", revokedTps, tps)
}
override def preStart(): Unit = {
super.preStart()
sourceActor = getStageActor(messageHandling)
log.info("Starting. StageActor {}", sourceActor.ref)
consumerActor = createConsumerActor()
sourceActor.watch(consumerActor)
configureSubscription(assignedCB, revokedCB)
}
protected def messageHandling: PartialFunction[(ActorRef, Any), Unit] = {
case (_, msg: KafkaConsumerActor.Internal.Messages[K @unchecked, V @unchecked]) =>
// might be more than one in flight when we assign/revoke tps
if (msg.requestId == requestId)
requested = false
buffer = buffer ++ msg.messages
pump()
case (_, Status.Failure(e)) =>
failStage(e)
case (_, Terminated(ref)) if ref == consumerActor =>
failStage(new ConsumerFailed())
}
protected def createConsumerActor(): ActorRef
override protected def configureManualSubscription(subscription: ManualSubscription): Unit = subscription match {
case Assignment(topics) =>
consumerActor.tell(KafkaConsumerActor.Internal.Assign(topics), sourceActor.ref)
tps ++= topics
case AssignmentWithOffset(topics) =>
consumerActor.tell(KafkaConsumerActor.Internal.AssignWithOffset(topics), sourceActor.ref)
tps ++= topics.keySet
case AssignmentOffsetsForTimes(topics) =>
consumerActor.tell(KafkaConsumerActor.Internal.AssignOffsetsForTimes(topics), sourceActor.ref)
tps ++= topics.keySet
}
@tailrec
private def pump(): Unit =
if (isAvailable(shape.out)) {
if (buffer.hasNext) {
val msg = buffer.next()
push(shape.out, createMessage(msg))
pump()
} else if (!requested && tps.nonEmpty) {
requestMessages()
}
}
protected def requestMessages(): Unit = {
requested = true
requestId += 1
log.debug("Requesting messages, requestId: {}, partitions: {}", requestId, tps)
consumerActor.tell(KafkaConsumerActor.Internal.RequestMessages(requestId, tps), sourceActor.ref)
}
setHandler(shape.out, new OutHandler {
override def onPull(): Unit = pump()
override def onDownstreamFinish(cause: Throwable): Unit =
performShutdown()
})
override def postStop(): Unit = {
onShutdown()
super.postStop()
}
def performShutdown(): Unit =
log.info("Completing")
}