-
Notifications
You must be signed in to change notification settings - Fork 27
/
kafka.clj
432 lines (383 loc) · 18.9 KB
/
kafka.clj
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
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
(ns onyx.plugin.kafka
(:require [onyx.plugin.partition-assignment :refer [partitions-for-slot]]
[onyx.kafka.helpers :as h]
[taoensso.timbre :as log :refer [fatal info]]
[onyx.static.default-vals :refer [arg-or-default]]
[onyx.plugin.protocols :as p]
[onyx.static.util :refer [kw->fn]]
[onyx.tasks.kafka]
[schema.core :as s])
(:import [java.util.concurrent.atomic AtomicLong]
[org.apache.kafka.clients.consumer ConsumerRecords ConsumerRecord]
[org.apache.kafka.clients.consumer KafkaConsumer ConsumerRebalanceListener Consumer]
[org.apache.kafka.common.record TimestampType]
[org.apache.kafka.common TopicPartition]
[org.apache.kafka.common.metrics Metrics]
[org.apache.kafka.clients.producer Callback KafkaProducer ProducerRecord]))
(def defaults
{:kafka/receive-buffer-bytes 65536
:kafka/wrap-with-metadata? false
:kafka/unable-to-find-broker-backoff-ms 8000})
(defn seek-offset! [log-prefix consumer kpartitions task-map topic checkpoint]
(let [policy (:kafka/offset-reset task-map)
start-offsets (:kafka/start-offsets task-map)]
(doseq [kpartition kpartitions]
(cond (get checkpoint kpartition)
(let [offset (get checkpoint kpartition)]
(info log-prefix "Seeking to checkpointed offset at:" (inc offset))
(h/seek-to-offset! consumer {:topic topic :partition kpartition} (inc offset)))
start-offsets
(let [offset (get start-offsets kpartition)]
(when-not offset
(throw (ex-info "Offset missing for existing partition when using :kafka/start-offsets"
{:missing-partition kpartition
:kafka/start-offsets start-offsets})))
(h/seek-to-offset! consumer {:topic topic :partition kpartition} offset))
(= policy :earliest)
(do
(info log-prefix "Seeking to earliest offset on topic" {:topic topic :partition kpartition})
(h/seek-to-beginning! consumer [{:topic topic :partition kpartition}]))
(= policy :latest)
(do
(info log-prefix "Seeking to latest offset on topic" {:topic topic :partition kpartition})
(h/seek-to-end! consumer [{:topic topic :partition kpartition}]))
:else
(throw (ex-info "Tried to seek to unknown policy" {:recoverable? false
:policy policy}))))))
(defn find-brokers [task-map]
(let [zk-addr (:kafka/zookeeper task-map)
_ (when-not zk-addr
(throw (ex-info "Either :kafka/bootstrap-servers or :kafka/zookeeper must be defined."
{:task-map task-map})))
results (vals (h/id->broker zk-addr))]
(if (seq results)
results
(do
(info "Could not locate any Kafka brokers to connect to. Backing off.")
(Thread/sleep (or (:kafka/unable-to-find-broker-backoff-ms task-map)
(:kafka/unable-to-find-broker-backoff-ms defaults)))
(throw (ex-info "Could not locate any Kafka brokers to connect to."
{:recoverable? true
:zk-addr zk-addr}))))))
(defprotocol PluginMeta
(metadata [this]))
(defn start-kafka-consumer
[event lifecycle]
{})
(defn check-num-peers-equals-partitions
[{:keys [onyx/min-peers onyx/max-peers onyx/n-peers kafka/partition] :as task-map} n-partitions]
(let [fixed-partition? (and partition (or (= 1 n-peers)
(= 1 max-peers)))
fixed-npeers? (or (and min-peers (= min-peers max-peers))
(= 1 max-peers)
(and n-peers (and (not min-peers) (not max-peers))))
n-peers (or max-peers n-peers)
n-peers-less-eq-n-partitions (and n-peers (<= n-peers n-partitions))]
(when-not (or fixed-partition? fixed-npeers? n-peers-less-eq-n-partitions)
(let [e (ex-info ":onyx/min-peers must equal :onyx/max-peers, or :onyx/n-peers must be set, and :onyx/min-peers and :onyx/max-peers must not be set. Number of peers should also be less than or equal to the number of partitions."
{:n-partitions n-partitions
:n-peers n-peers
:min-peers min-peers
:max-peers max-peers
:recoverable? false
:task-map task-map})]
(log/error e)
(throw e)))))
(defn assign-partitions-to-slot! [consumer* task-map topic n-partitions slot]
(if-let [part (:partition task-map)]
(let [p (Integer/parseInt part)]
(h/assign-partitions! consumer* [{:topic topic :partition p}])
[p])
(let [n-slots (or (:onyx/n-peers task-map) (:onyx/max-peers task-map))
[lower upper] (partitions-for-slot n-partitions n-slots slot)
parts-range (range lower (inc upper))
parts (map (fn [p] {:topic topic :partition p}) parts-range)]
(h/assign-partitions! consumer* parts)
parts-range)))
(defn set-lag! [^AtomicLong lag-gauge ^KafkaConsumer consumer]
(.set lag-gauge
(reduce (fn [lag [tp offset]]
(+ lag (- offset (.position consumer tp))))
0
(.endOffsets consumer (.assignment consumer)))))
(defn all-partitions-paused?
[^KafkaConsumer consumer kpartitions]
(let [paused (into #{}
(map #(.partition ^TopicPartition %))
(.paused consumer))]
(= paused (set kpartitions))))
(defn paused? [^KafkaConsumer consumer part]
(let [paused (map #(.partition ^TopicPartition %)
(.paused consumer))]
(some #{part} paused)))
(defn current-partition-statuses [checkpoint ^KafkaConsumer consumer target-offsets kpartitions]
(if target-offsets
(let [beginning-offsets (into {}
(map (fn [[^TopicPartition topic-partition offset]]
[(.partition topic-partition) offset])
(.beginningOffsets consumer (.assignment consumer))))]
(->> kpartitions
(map (fn [p]
(let [current-offset (or (get checkpoint p) (get beginning-offsets p))
_ (assert current-offset)
target-offset (get target-offsets p)
drained? (and target-offset
(>= current-offset target-offset))]
[p (if drained? :drained :reading)])))
(into {})))
(->> kpartitions
(map (fn [p] [p :reading]))
(into {}))))
(deftype KafkaReadMessages
[log-prefix task-map topic ^:unsynchronized-mutable kpartitions batch-timeout
deserializer-fn segment-fn ^AtomicLong watermark ^AtomicLong lag-gauge ^KafkaConsumer ^:unsynchronized-mutable consumer
^:unsynchronized-mutable iter ^:unsynchronized-mutable partition->offset drained
target-offsets]
PluginMeta
(metadata [this]
{:consumer consumer
:beginning-offsets (into {}
(map (fn [[^TopicPartition topic-partition offset]]
[{:topic (.topic topic-partition)
:partition (.partition topic-partition)}
offset])
(.beginningOffsets consumer (.assignment consumer))))
:end-offsets (into {}
(map (fn [[^TopicPartition topic-partition offset]]
[{:topic (.topic topic-partition)
:partition (.partition topic-partition)}
offset])
(.endOffsets consumer (.assignment consumer))))
:partitions kpartitions
:partition->offset partition->offset})
p/Plugin
(start [this event]
(let [{:keys [kafka/bootstrap-servers kafka/group-id kafka/consumer-opts]} task-map
brokers (or bootstrap-servers (find-brokers task-map))
_ (s/validate onyx.tasks.kafka/KafkaInputTaskMap task-map)
consumer-config (merge {"bootstrap.servers" brokers
"group.id" (or group-id "onyx")
"enable.auto.commit" false
"receive.buffer.bytes" (or (:kafka/receive-buffer-bytes task-map)
(:kafka/receive-buffer-bytes defaults))
"auto.offset.reset" (name (:kafka/offset-reset task-map))}
consumer-opts)
_ (info log-prefix "Starting kafka/read-messages task with consumer opts:" consumer-config)
key-deserializer (h/byte-array-deserializer)
value-deserializer (h/byte-array-deserializer)
consumer* (h/build-consumer consumer-config key-deserializer value-deserializer)
_ (when (and (:kafka/target-offsets task-map)
(:kafka/start-offsets task-map)
(not= (set (keys (:kafka/target-offsets task-map)))
(set (keys (:kafka/start-offsets task-map)))))
(throw (ex-info "When :kafka/start-offsets and :kafka/target-offsets are both specified, they must both contain the same partitions."
task-map)))
partitions (mapv :partition
(or (keys (:kafka/target-offsets task-map))
(keys (:kafka/start-offsets task-map))
(h/partitions-for-topic consumer* topic)))
n-partitions (count partitions)]
(check-num-peers-equals-partitions task-map n-partitions)
(let [kpartitions* (assign-partitions-to-slot! consumer* task-map topic n-partitions (:onyx.core/slot-id event))]
(set! consumer consumer*)
(set! kpartitions kpartitions*)
this)))
(stop [this event]
(when consumer
(.close consumer)
(set! consumer nil))
this)
p/WatermarkedInput
(watermark [this]
(.get watermark))
p/Checkpointed
(checkpoint [this]
partition->offset)
;; checkpoint map looks like {part offset}
(recover! [this replica-version checkpoint]
(let [partition-statuses (current-partition-statuses checkpoint consumer target-offsets kpartitions)
resuming-tps (reduce-kv
(fn [all part v]
(if (not (some #{v} #{:emitted :drained}))
(conj all (TopicPartition. topic part))
all))
[]
partition-statuses)]
(.resume consumer resuming-tps)
(reset! drained partition-statuses)
(set! iter nil)
(set! partition->offset checkpoint)
(seek-offset! log-prefix consumer kpartitions task-map topic checkpoint))
this)
(checkpointed! [this epoch])
p/BarrierSynchronization
(synced? [this epoch]
(set-lag! lag-gauge consumer)
true)
(completed? [this]
(empty? (remove #(= :emitted %) (vals @drained))))
p/Input
(poll! [this _ remaining-ms]
(if-let [drained-part (ffirst (filter (fn [[_ state]] (= state :drained)) @drained))]
(do (swap! drained assoc drained-part :emitted)
{:type :end-reached :partition drained-part})
(if (and iter (.hasNext ^java.util.Iterator iter))
(let [rec ^ConsumerRecord (.next ^java.util.Iterator iter)
deserialized (some-> rec segment-fn)
part (.partition rec)]
(.set watermark (max (.get watermark) (.timestamp rec)))
;; TODO: remove support for :done in favor of target-offsets
(cond (= :done deserialized)
(do (swap! drained assoc (.partition rec) :emitted)
nil)
deserialized
(let [new-offset (.offset rec)]
(set! partition->offset (assoc partition->offset part new-offset))
(if-let [target-offset (get target-offsets part)]
(let [tp (TopicPartition. topic part)]
(if (>= new-offset target-offset)
(when (not (paused? consumer part))
(.pause consumer [tp])
(swap! drained assoc (.partition rec) :drained)
;; only emit message if it's on the boundary
(if (= new-offset target-offset)
deserialized))
deserialized))
deserialized))))
(do (set! iter (.iterator ^ConsumerRecords (.poll ^Consumer consumer remaining-ms)))
nil)))))
(defn read-messages [{:keys [onyx.core/task-map onyx.core/log-prefix onyx.core/monitoring]}]
(let [{:keys [kafka/topic kafka/deserializer-fn kafka/target-offsets]} task-map
batch-timeout (arg-or-default :onyx/batch-timeout task-map)
wrap-message? (or (:kafka/wrap-with-metadata? task-map) (:kafka/wrap-with-metadata? defaults))
deserializer-fn (kw->fn (:kafka/deserializer-fn task-map))
key-deserializer-fn (if-let [kw (:kafka/key-deserializer-fn task-map)] (kw->fn kw) identity)
segment-fn (if wrap-message?
(fn [^ConsumerRecord cr]
{:topic (.topic cr)
:partition (.partition cr)
:key (when-let [k (.key cr)] (key-deserializer-fn k))
:message (deserializer-fn (.value cr))
:serialized-key-size (.serializedKeySize cr)
:serialized-value-size (.serializedValueSize cr)
:timestamp (.timestamp cr)
:timestamp-type (.id ^TimestampType (.timestampType cr))
:offset (.offset cr)})
(fn [^ConsumerRecord cr]
(deserializer-fn (.value cr))))
watermark (AtomicLong. 0)
{:keys [lag-gauge]} monitoring]
(->KafkaReadMessages log-prefix task-map topic nil batch-timeout
deserializer-fn segment-fn watermark lag-gauge
nil nil nil (atom {}) target-offsets)))
(defn close-read-messages
[event lifecycle]
{})
(defn inject-write-messages
[event lifecycle]
{})
(defn close-write-resources
[event lifecycle]
{})
(defn- message->producer-record
[key-serializer-fn serializer-fn topic kpartition m]
(let [message (:message m)
k (some-> m :key key-serializer-fn)
message-topic (get m :topic topic)
message-partition (some-> m (get :partition kpartition) int)
message-timestamp (some-> m (get :timestamp) long)]
(cond (not (contains? m :message))
(throw (ex-info "Payload is missing required. Need message key :message"
{:recoverable? false
:payload m}))
(nil? message-topic)
(throw (ex-info
(str "Unable to write message payload to Kafka! "
"Both :kafka/topic, and :topic in message payload "
"are missing!")
{:recoverable? false
:payload m}))
:else
(ProducerRecord. ^String message-topic ^Integer message-partition ^Long message-timestamp k (serializer-fn message)))))
(defn clear-write-futures! [fs]
(doall (remove (fn [^java.util.concurrent.Future f]
(assert (not (.isCancelled f)))
(.isDone f))
fs)))
(defrecord KafkaWriteMessages [task-map config topic kpartition producer key-serializer-fn serializer-fn write-futures exception write-callback]
p/Plugin
(start [this event]
this)
(stop [this event]
(.close ^KafkaProducer producer)
this)
p/BarrierSynchronization
(synced? [this epoch]
(when @exception (throw @exception))
(empty? (vswap! write-futures clear-write-futures!)))
(completed? [this]
(when @exception (throw @exception))
(empty? (vswap! write-futures clear-write-futures!)))
p/Checkpointed
(recover! [this _ _]
this)
(checkpoint [this])
(checkpointed! [this epoch])
p/Output
(prepare-batch [this event replica _]
true)
(write-batch [this {:keys [onyx.core/write-batch]} replica _]
(when @exception (throw @exception))
(vswap! write-futures
(fn [fs]
(-> fs
(clear-write-futures!)
(into (map
(fn [msg]
(let [record (message->producer-record key-serializer-fn serializer-fn topic kpartition msg)]
(.send ^KafkaProducer producer record write-callback)))
write-batch)))))
true))
(def write-defaults {:kafka/request-size 307200})
(deftype ExceptionCallback [e]
Callback
(onCompletion [_ v exception]
(when exception (reset! e exception))))
(defn write-messages [{:keys [onyx.core/task-map onyx.core/log-prefix]}]
(let [_ (s/validate onyx.tasks.kafka/KafkaOutputTaskMap task-map)
brokers (or (:kafka/bootstrap-servers task-map) (find-brokers task-map))
request-size (or (get task-map :kafka/request-size) (get write-defaults :kafka/request-size))
producer-opts (:kafka/producer-opts task-map)
config (merge {"bootstrap.servers" brokers
"max.request.size" request-size}
producer-opts)
_ (info log-prefix "Starting kafka/write-messages task with producer opts:" config)
topic (:kafka/topic task-map)
kpartition (:kafka/partition task-map)
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)
producer (h/build-producer config key-serializer value-serializer)
serializer-fn (kw->fn (:kafka/serializer-fn task-map))
key-serializer-fn (if-let [kw (:kafka/key-serializer-fn task-map)] (kw->fn kw) identity)
exception (atom nil)
write-callback (->ExceptionCallback exception)
write-futures (volatile! (list))]
(->KafkaWriteMessages task-map config topic kpartition producer
key-serializer-fn serializer-fn
write-futures exception write-callback)))
(defn read-handle-exception [event lifecycle lf-kw exception]
(if (false? (:recoverable? (ex-data exception)))
:kill
:restart))
(def read-messages-calls
{:lifecycle/before-task-start start-kafka-consumer
:lifecycle/handle-exception read-handle-exception
:lifecycle/after-task-stop close-read-messages})
(defn write-handle-exception [event lifecycle lf-kw exception]
(if (false? (:recoverable? (ex-data exception)))
:kill
:restart))
(def write-messages-calls
{:lifecycle/before-task-start inject-write-messages
:lifecycle/handle-exception write-handle-exception
:lifecycle/after-task-stop close-write-resources})