-
Notifications
You must be signed in to change notification settings - Fork 97
/
CommittableOffsetBatch.scala
307 lines (272 loc) · 10.9 KB
/
CommittableOffsetBatch.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
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
/*
* Copyright 2018-2020 OVO Energy Limited
*
* SPDX-License-Identifier: Apache-2.0
*/
package fs2.kafka
import cats.ApplicativeError
import cats.instances.list._
import cats.syntax.foldable._
import cats.syntax.show._
import cats.{Applicative, Foldable, Show}
import fs2.kafka.internal.instances._
import fs2.kafka.internal.syntax._
import org.apache.kafka.clients.consumer.OffsetAndMetadata
import org.apache.kafka.common.TopicPartition
/**
* [[CommittableOffsetBatch]] represents a batch of Kafka [[offsets]]
* which can be committed together using [[commit]]. An offset, or one
* more batch, can be added an existing batch using `updated`. Note that
* this requires the offsets per topic-partition to be included in-order,
* since offset commits in general require it.<br>
* <br>
* Use [[CommittableOffsetBatch#empty]] to create an empty batch. The
* [[CommittableOffset#batch]] function can be used to create a batch
* from an existing [[CommittableOffset]].<br>
* <br>
* If you have some offsets in-order per topic-partition, you can fold
* them together using [[CommittableOffsetBatch#empty]] and `updated`,
* or you can use [[CommittableOffsetBatch#fromFoldable]]. Generally,
* prefer to use `fromFoldable`, as it has better performance. Provided
* pipes like [[commitBatchWithin]] are also to be preferred, as they
* also achieve better performance.
*/
sealed abstract class CommittableOffsetBatch[F[_]] {
/**
* Creates a new [[CommittableOffsetBatch]] with the specified offset
* included. Note that this function requires offsets to be in-order
* per topic-partition, as provided offsets will override existing
* offsets for the same topic-partition.
*/
def updated(that: CommittableOffset[F]): CommittableOffsetBatch[F]
/**
* Creates a new [[CommittableOffsetBatch]] with the specified offsets
* included. Note that this function requires offsets to be in-order
* per topic-partition, as provided offsets will override existing
* offsets for the same topic-partition.
*/
def updated(that: CommittableOffsetBatch[F]): CommittableOffsetBatch[F]
/**
* The offsets included in the [[CommittableOffsetBatch]].
*/
def offsets: Map[TopicPartition, OffsetAndMetadata]
/**
* The consumer group IDs for the [[offsets]] in the batch.
* For the batch to be valid and for [[commit]] to succeed,
* there should be exactly one ID in the set and the flag
* [[consumerGroupIdsMissing]] should be `false`.<br>
* <br>
* There might be more than one consumer group ID in the set
* if offsets from multiple consumers, with different group
* IDs, have accidentally been mixed. The set might also be
* empty if no consumer group IDs have been specified.
*/
def consumerGroupIds: Set[String]
/**
* `true` if any offset in the batch came from a consumer
* without a group ID; `false` otherwise. For the batch to
* be valid and for [[commit]] to succeed, this flag must
* be `false` and there should be exactly one consumer
* group ID in [[consumerGroupIds]].
*/
def consumerGroupIdsMissing: Boolean
/**
* Commits the [[offsets]] to Kafka in a single commit.
* For the batch to be valid and for commit to succeed,
* the following conditions must hold:<br>
* - [[consumerGroupIdsMissing]] must be false, and<br>
* - [[consumerGroupIds]] must have exactly one ID.<br>
* <br>
* If one of the conditions above do not hold, there will
* be a [[ConsumerGroupException]] exception raised and a
* commit will not be attempted. If [[offsets]] is empty
* then these conditions do not need to hold, as there
* is nothing to commit.
*/
def commit: F[Unit]
}
object CommittableOffsetBatch {
private[kafka] def apply[F[_]](
offsets: Map[TopicPartition, OffsetAndMetadata],
consumerGroupIds: Set[String],
consumerGroupIdsMissing: Boolean,
commit: Map[TopicPartition, OffsetAndMetadata] => F[Unit]
)(implicit F: ApplicativeError[F, Throwable]): CommittableOffsetBatch[F] = {
val _offsets = offsets
val _consumerGroupIds = consumerGroupIds
val _consumerGroupIdsMissing = consumerGroupIdsMissing
val _commit = commit
new CommittableOffsetBatch[F] {
override def updated(that: CommittableOffset[F]): CommittableOffsetBatch[F] =
CommittableOffsetBatch(
_offsets.updated(that.topicPartition, that.offsetAndMetadata),
that.consumerGroupId.fold(_consumerGroupIds)(_consumerGroupIds + _),
_consumerGroupIdsMissing || that.consumerGroupId.isEmpty,
_commit
)
override def updated(that: CommittableOffsetBatch[F]): CommittableOffsetBatch[F] =
CommittableOffsetBatch(
_offsets ++ that.offsets,
_consumerGroupIds ++ that.consumerGroupIds,
_consumerGroupIdsMissing || that.consumerGroupIdsMissing,
_commit
)
override val offsets: Map[TopicPartition, OffsetAndMetadata] =
_offsets
override val consumerGroupIds: Set[String] =
_consumerGroupIds
override val consumerGroupIdsMissing: Boolean =
_consumerGroupIdsMissing
override def commit: F[Unit] =
if (_consumerGroupIdsMissing || _consumerGroupIds.size != 1)
F.raiseError(ConsumerGroupException(consumerGroupIds))
else _commit(offsets)
override def toString: String =
Show[CommittableOffsetBatch[F]].show(this)
}
}
/**
* Creates a [[CommittableOffsetBatch]] from some [[CommittableOffset]]s,
* where the containing type has a `Foldable` instance. Guaranteed to be
* equivalent to the following, but implemented more efficiently.
*
* {{{
* offsets.foldLeft(CommittableOffsetBatch.empty[F])(_ updated _)
* }}}
*
* Note that just like for `updated`, `offsets` have to be in order
* per topic-partition.
*
* @see [[CommittableOffsetBatch#fromFoldableMap]]
* @see [[CommittableOffsetBatch#fromFoldableOption]]
*/
def fromFoldable[F[_], G[_]](offsets: G[CommittableOffset[F]])(
implicit F: ApplicativeError[F, Throwable],
G: Foldable[G]
): CommittableOffsetBatch[F] =
fromFoldableMap(offsets)(identity)
/**
* Creates a [[CommittableOffsetBatch]] from a `Foldable` containing
* `A`s, by applying `f` to each `A` to get the [[CommittableOffset]].
* Guaranteed to be equivalent to the following, but implemented more
* efficiently.
*
* {{{
* ga.foldLeft(CommittableOffsetBatch.empty[F])(_ updated f(_))
* }}}
*
* Note that just like for `updated`, `offsets` have to be in order
* per topic-partition.
*
* @see [[CommittableOffsetBatch#fromFoldable]]
* @see [[CommittableOffsetBatch#fromFoldableOption]]
*/
def fromFoldableMap[F[_], G[_], A](ga: G[A])(f: A => CommittableOffset[F])(
implicit F: ApplicativeError[F, Throwable],
G: Foldable[G]
): CommittableOffsetBatch[F] = {
var commit: Map[TopicPartition, OffsetAndMetadata] => F[Unit] = null
var offsetsMap: Map[TopicPartition, OffsetAndMetadata] = Map.empty
var consumerGroupIds: Set[String] = Set.empty
var consumerGroupIdsMissing: Boolean = false
var empty: Boolean = true
ga.foldLeft(()) { (_, a) =>
val offset = f(a)
if (empty) {
commit = offset.commitOffsets
empty = false
}
offsetsMap = offsetsMap.updated(offset.topicPartition, offset.offsetAndMetadata)
offset.consumerGroupId match {
case Some(consumerGroupId) => consumerGroupIds = consumerGroupIds + consumerGroupId
case None => consumerGroupIdsMissing = true
}
}
if (empty) CommittableOffsetBatch.empty[F]
else CommittableOffsetBatch(offsetsMap, consumerGroupIds, consumerGroupIdsMissing, commit)
}
/**
* Creates a [[CommittableOffsetBatch]] from some [[CommittableOffset]]s wrapped
* in `Option`, where the containing type has a `Foldable` instance. Guaranteed
* to be equivalent to the following, but implemented more efficiently.
*
* {{{
* offsets.foldLeft(CommittableOffsetBatch.empty[F]) {
* case (batch, Some(offset)) => batch.updated(offset)
* case (batch, None) => batch
* }
* }}}
*
* Note that just like for `updated`, `offsets` have to be in order
* per topic-partition.
*
* @see [[CommittableOffsetBatch#fromFoldable]]
* @see [[CommittableOffsetBatch#fromFoldableMap]]
*/
def fromFoldableOption[F[_], G[_]](offsets: G[Option[CommittableOffset[F]]])(
implicit F: ApplicativeError[F, Throwable],
G: Foldable[G]
): CommittableOffsetBatch[F] = {
var commit: Map[TopicPartition, OffsetAndMetadata] => F[Unit] = null
var offsetsMap: Map[TopicPartition, OffsetAndMetadata] = Map.empty
var consumerGroupIds: Set[String] = Set.empty
var consumerGroupIdsMissing: Boolean = false
var empty: Boolean = true
offsets.foldLeft(()) {
case (_, Some(offset)) =>
if (empty) {
commit = offset.commitOffsets
empty = false
}
offsetsMap = offsetsMap.updated(offset.topicPartition, offset.offsetAndMetadata)
offset.consumerGroupId match {
case Some(consumerGroupId) => consumerGroupIds = consumerGroupIds + consumerGroupId
case None => consumerGroupIdsMissing = true
}
case (_, None) => ()
}
if (empty) CommittableOffsetBatch.empty[F]
else CommittableOffsetBatch(offsetsMap, consumerGroupIds, consumerGroupIdsMissing, commit)
}
/**
* An empty [[CommittableOffsetBatch]] which does not include any
* offsets and `commit` will not commit offsets. This can be used
* together with `updated` to create a batch from some offsets.
*
* @see [[CommittableOffsetBatch#fromFoldable]]
* @see [[CommittableOffsetBatch#fromFoldableOption]]
*/
def empty[F[_]](implicit F: Applicative[F]): CommittableOffsetBatch[F] =
new CommittableOffsetBatch[F] {
override def updated(that: CommittableOffset[F]): CommittableOffsetBatch[F] =
that.batch
override def updated(that: CommittableOffsetBatch[F]): CommittableOffsetBatch[F] =
that
override val offsets: Map[TopicPartition, OffsetAndMetadata] =
Map.empty
override val consumerGroupIds: Set[String] =
Set.empty
override val consumerGroupIdsMissing: Boolean =
false
override val commit: F[Unit] =
F.unit
override def toString: String =
Show[CommittableOffsetBatch[F]].show(this)
}
implicit def committableOffsetBatchShow[F[_]]: Show[CommittableOffsetBatch[F]] =
Show.show { cob =>
if (cob.offsets.isEmpty) "CommittableOffsetBatch(<empty>)"
else {
cob.offsets.toList.sorted.mkStringAppend {
case (append, (tp, oam)) =>
append(tp.show)
append(" -> ")
append(oam.show)
}(
start = "CommittableOffsetBatch(",
sep = ", ",
end = ")"
)
}
}
}