-
Notifications
You must be signed in to change notification settings - Fork 27
/
helpers.clj
194 lines (165 loc) · 7.12 KB
/
helpers.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
(ns onyx.kafka.helpers
(:require [clojure.string :as string]
[taoensso.timbre :as log])
(:import [kafka.utils ZkUtils]
[org.apache.kafka.clients.consumer KafkaConsumer ConsumerRecord]
[org.apache.kafka.clients.producer KafkaProducer Callback ProducerRecord]
[org.apache.kafka.common.serialization ByteArrayDeserializer ByteArraySerializer Serializer Deserializer]
[org.apache.kafka.common TopicPartition PartitionInfo]
[kafka.admin AdminUtils]
[kafka.utils ZKStringSerializer$]
[org.I0Itec.zkclient ZkClient ZkConnection IZkConnection]
[org.I0Itec.zkclient.serialize ZkSerializer]
[scala.collection JavaConversions]
[java.util Properties]))
(def zk-defaults
{:session-timeout 30000
:connection-timeout 30000
:operation-retry-timeout (long -1)
:serializer (ZKStringSerializer$/MODULE$)})
(defn munge-servers [servers]
(if (coll? servers)
(string/join "," servers)
servers))
(defn as-properties ^Properties [m]
(let [ps (Properties.)]
(doseq [[k v] m] (.setProperty ps k (str v)))
ps))
(defn as-java [opts]
(cond-> opts
(get opts "bootstrap.servers") (update "bootstrap.servers" munge-servers)))
(defn make-zk-connection [zk-config]
(let [{:keys [servers session-timeout]} zk-config
server-seq (if (coll? servers) (string/join "," servers) servers)]
(ZkConnection. server-seq session-timeout)))
(defn make-zk-client [zk-connection zk-config]
(let [{:keys [connection-timeout operation-retry-timeout serializer]} zk-config]
(ZkClient. ^IZkConnection zk-connection
(int connection-timeout)
^ZkSerializer serializer
(long operation-retry-timeout))))
(defn make-zk-utils ^ZkUtils [zk-config secure?]
(let [merged-config (merge zk-defaults zk-config)
session-timeout (:session-timeout merged-config)
connection-timeout (:connection-timeout merged-config)
zk-connection (make-zk-connection merged-config)
zk-client (make-zk-client zk-connection merged-config)]
(ZkUtils. zk-client zk-connection secure?)))
(defn all-brokers
[^ZkUtils zk-utils]
(->> (.getAllBrokersInCluster zk-utils)
(JavaConversions/bufferAsJavaList)
(map
(fn [broker]
{:id (.id broker)
:endpoints
(map
(fn [endpoint]
{:host (.host endpoint)
:port (.port endpoint)
:protocol-type (.name (.securityProtocol endpoint))})
(JavaConversions/seqAsJavaList (.endPoints broker)))}))))
(defn id->broker [zk-addr]
(with-open [zk-utils (make-zk-utils {:servers zk-addr} false)]
(reduce
(fn [result {:keys [id endpoints]}]
(assoc
result
id
(str (:host (first endpoints)) ":" (:port (first endpoints)))))
{}
(all-brokers zk-utils))))
(defn byte-array-serializer []
(ByteArraySerializer.))
(defn byte-array-deserializer []
(ByteArrayDeserializer.))
(defn ^KafkaProducer build-producer [producer-opts key-serializer value-serializer]
(KafkaProducer. ^Properties (as-properties (as-java producer-opts))
^Serializer key-serializer
^Serializer value-serializer))
(defn ^KafkaConsumer build-consumer [consumer-opts key-deserializer value-deserializer]
(KafkaConsumer. ^Properties (as-properties (as-java consumer-opts))
^Deserializer key-deserializer
^Deserializer value-deserializer))
(defn partitions-for-topic [consumer topic]
(let [parts (.partitionsFor ^KafkaConsumer consumer topic)]
(map
(fn [part]
{:topic (.topic part)
:partition (.partition part)})
parts)))
(defn to-topic-partition [tp]
(TopicPartition. (:topic tp) (:partition tp)))
(defn assign-partitions! [consumer topic-partitions]
(->> topic-partitions
(mapv to-topic-partition)
(.assign ^KafkaConsumer consumer)))
(defn seek-to-offset! [consumer topic-partition offset]
(let [encoded (to-topic-partition topic-partition)]
(.seek ^KafkaConsumer consumer encoded offset)))
(defn seek-to-beginning! [consumer topic-partitions]
(let [encoded (map to-topic-partition topic-partitions)]
(.seekToBeginning ^KafkaConsumer consumer encoded)))
(defn seek-to-end! [consumer topic-partitions]
(let [encoded (map to-topic-partition topic-partitions)]
(.seekToEnd ^KafkaConsumer consumer encoded)))
(defn consumer-record->message
[decompress-fn ^ConsumerRecord m]
{:key (some-> m (.key) decompress-fn)
:partition (.partition m)
:topic (.topic m)
:value (-> m (.value) decompress-fn)
:timestamp (.timestamp m)})
(defn poll! [consumer timeout]
(.poll ^KafkaConsumer consumer timeout))
(defn take-now
"Reads whatever it can from a topic on the assumption that we've distributed
work across multiple topics and another topic contained :done."
([bootstrap-servers topic decompress-fn]
(take-now bootstrap-servers topic decompress-fn 5000))
([bootstrap-servers topic decompress-fn timeout]
(log/info {:msg "Taking now..." :topic topic})
(let [c (build-consumer {"bootstrap.servers" bootstrap-servers} (byte-array-deserializer) (byte-array-deserializer))
topic-partitions [{:topic topic :partition 0}]]
(assign-partitions! c topic-partitions)
(seek-to-beginning! c topic-partitions)
(mapv #(consumer-record->message decompress-fn %) (poll! c timeout)))))
(defn create-topic! [zk-addr topic-name num-partitions replication-factor]
(with-open [zk-utils (make-zk-utils {:servers zk-addr} false)]
(AdminUtils/createTopic
zk-utils topic-name num-partitions replication-factor (as-properties {})
(kafka.admin.RackAwareMode$Safe$.))))
(deftype ProducerCallback [p]
Callback
(onCompletion [_ v exception]
(deliver p true)))
(defn send-sync! [producer topic part k v]
(let [p (promise)
record (ProducerRecord. topic part k v)]
(.send producer record (->ProducerCallback p))
@p))
(defn partition-info->topic-partition [topic ^PartitionInfo part-info]
(TopicPartition. topic (.partition part-info)))
(defn end-offsets [bootstrap-servers topic]
(let [opts {"bootstrap.servers" bootstrap-servers}
k-deser (ByteArrayDeserializer.)
v-deser (ByteArrayDeserializer.)]
(with-open [consumer (build-consumer opts k-deser v-deser)]
(let [parts (.partitionsFor consumer topic)
tps (map (partial partition-info->topic-partition topic) parts)]
(.endOffsets consumer tps)))))
(defn offsets->clj [end-offsets]
(reduce-kv
(fn [all ^TopicPartition k v]
(assoc all (.partition k) v))
{}
(into {} end-offsets)))
(defn beginning-end-offsets-clj [bootstrap-servers topic]
(let [opts {"bootstrap.servers" bootstrap-servers}
k-deser (ByteArrayDeserializer.)
v-deser (ByteArrayDeserializer.)]
(with-open [consumer (build-consumer opts k-deser v-deser)]
(let [parts (.partitionsFor consumer topic)
tps (map (partial partition-info->topic-partition topic) parts)]
{:beginning-offsets (offsets->clj (.beginningOffsets consumer tps))
:end-offsets (offsets->clj (.endOffsets consumer tps))}))))