Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Custom serialization #61

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
86 changes: 52 additions & 34 deletions src/onyx/kafka/helpers.clj
Original file line number Diff line number Diff line change
Expand Up @@ -79,21 +79,27 @@
{}
(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 byte-array-serializer-name []
(.getName ^Class ByteArraySerializer))

(defn byte-array-deserializer-name []
(.getName ^Class ByteArrayDeserializer))

(defn ^KafkaProducer build-producer
([producer-opts]
(KafkaProducer. ^Properties (as-properties (as-java producer-opts))))
([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]
(KafkaConsumer. ^Properties (as-properties (as-java consumer-opts))))
([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)]
Expand Down Expand Up @@ -138,10 +144,16 @@
"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))
(take-now bootstrap-servers topic decompress-fn 5000 (byte-array-deserializer-name) (byte-array-deserializer-name)))
([bootstrap-servers topic decompress-fn timeout]
(take-now bootstrap-servers topic decompress-fn timeout (byte-array-deserializer-name) (byte-array-deserializer-name)))
([bootstrap-servers topic decompress-fn key-deserializer-name value-deserializer-name]
(take-now bootstrap-servers topic decompress-fn 5000 key-deserializer-name value-deserializer-name))
([bootstrap-servers topic decompress-fn timeout key-deserializer-name value-deserializer-name]
(log/info {:msg "Taking now..." :topic topic})
(let [c (build-consumer {"bootstrap.servers" bootstrap-servers} (byte-array-deserializer) (byte-array-deserializer))
(let [c (build-consumer {"bootstrap.servers" bootstrap-servers
"key.deserializer" key-deserializer-name
"value.deserializer" value-deserializer-name})
topic-partitions [{:topic topic :partition 0}]]
(assign-partitions! c topic-partitions)
(seek-to-beginning! c topic-partitions)
Expand All @@ -167,14 +179,17 @@
(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 end-offsets
([bootstrap-servers topic]
(end-offsets bootstrap-servers topic (byte-array-deserializer-name) (byte-array-deserializer-name)))
([bootstrap-servers topic key-deserializer-name value-deserializer-name]
(let [opts {"bootstrap.servers" bootstrap-servers
"key.deserializer" key-deserializer-name
"value.deserializer" value-deserializer-name}]
(with-open [consumer (build-consumer opts)]
(let [parts (.partitionsFor consumer topic)
tps (map (partial partition-info->topic-partition topic) parts)]
(.endOffsets consumer tps))))))

(defn offsets->clj [end-offsets]
(reduce-kv
Expand All @@ -183,12 +198,15 @@
{}
(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))}))))
(defn beginning-end-offsets-clj
([bootstrap-servers topic]
(beginning-end-offsets-clj bootstrap-servers topic (byte-array-deserializer-name) (byte-array-deserializer-name)))
([bootstrap-servers topic key-deserializer-name value-deserializer-name]
(let [opts {"bootstrap.servers" bootstrap-servers
"key.deserializer" key-deserializer-name
"value.deserializer" value-deserializer-name}]
(with-open [consumer (build-consumer opts)]
(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))})))))
32 changes: 30 additions & 2 deletions src/onyx/kafka/information_model.cljc
Original file line number Diff line number Diff line change
Expand Up @@ -79,12 +79,24 @@
:default 2000
:optional? true}

:kafka/deserializer
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:kafka/deserializer -> :kafka/value-deserializer for consistency with kafka.

Copy link
Author

@danieroux danieroux Nov 21, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one is a tricky one for me.

I was keeping it consistent with :kafka/deserializer-fn - if you change the :kafka/deserializer you have to consider how it affects your :kafka/deserializer-fnchoice.

I changed the documentation to say something about it.

Do you still want me to change it to be consistent with kafka?

{:doc "Set the 'value.deserializer' property of KafkaConsumer."
:default "org.apache.kafka.common.serialization.ByteArrayDeserializer"
:type :string
:optional? true}

:kafka/deserializer-fn
{:doc "A keyword that represents a fully qualified namespaced function to deserialize a record's value. Takes one argument, which must be a byte array."
{:doc "A keyword that represents a fully qualified namespaced function to deserialize a record's value. Takes one argument, which must be a byte array (if :kafka/deserializer is the default)"
:type :keyword}

:kafka/key-deserializer
{:doc "Set the 'key.deserializer' property of KafkaConsumer."
:default "org.apache.kafka.common.serialization.ByteArrayDeserializer"
:type :string
:optional? true}

:kafka/key-deserializer-fn
{:doc "A keyword that represents a fully qualified namespaced function to deserialize a record's key. Takes one argument, which must be a byte array. Only used when `:kafka/wrap-with-metadata?` is true."
{:doc "A keyword that represents a fully qualified namespaced function to deserialize a record's key. Takes one argument, which must be a byte array (if :kafka/key-deserializer is the default). Only used when `:kafka/wrap-with-metadata?` is true."
:type :keyword
:optional? true}

Expand Down Expand Up @@ -131,10 +143,22 @@
:type :long
:optional? true}

:kafka/serializer
{:doc "Set the 'value.serializer' property of KafkaProducer."
:default "org.apache.kafka.common.serialization.ByteArraySerializer"
:type :string
:optional? true}

:kafka/serializer-fn
{:doc "A keyword that represents a fully qualified namespaced function to serialize a record's value. Takes one argument - the segment."
:type :keyword}

:kafka/key-serializer
{:doc "Set the 'key.serializer' property of KafkaProducer."
:default "org.apache.kafka.common.serialization.ByteArraySerializer"
:type :string
:optional? true}

:kafka/key-serializer-fn
{:doc "A keyword that represents a fully qualified namespaced function to serialize a record's key. Takes one argument - the segment."
:type :keyword
Expand Down Expand Up @@ -171,7 +195,9 @@
:kafka/bootstrap-servers
:kafka/offset-reset
:kafka/force-reset?
:kafka/deserializer
:kafka/deserializer-fn
:kafka/key-deserializer
:kafka/key-deserializer-fn
:kafka/receive-buffer-bytes
:kafka/commit-interval
Expand All @@ -188,7 +214,9 @@
:kafka/zookeeper
:kafka/bootstrap-servers
:kafka/partition
:kafka/serializer
:kafka/serializer-fn
:kafka/key-serializer
:kafka/key-serializer-fn
:kafka/request-size
:kafka/no-seal?
Expand Down
26 changes: 17 additions & 9 deletions src/onyx/plugin/kafka.clj
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,11 @@
(def defaults
{:kafka/receive-buffer-bytes 65536
:kafka/wrap-with-metadata? false
:kafka/unable-to-find-broker-backoff-ms 8000})
:kafka/unable-to-find-broker-backoff-ms 8000
:kafka/key-deserializer (h/byte-array-deserializer-name)
:kafka/deserializer (h/byte-array-deserializer-name)
:kafka/key-serializer (h/byte-array-serializer-name)
:kafka/serializer (h/byte-array-serializer-name)})

(defn seek-offset! [log-prefix consumer kpartitions task-map topic checkpoint]
(let [policy (:kafka/offset-reset task-map)
Expand Down Expand Up @@ -176,12 +180,14 @@
"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))}
"auto.offset.reset" (name (:kafka/offset-reset task-map))
"key.deserializer" (or (:kafka/key-deserializer task-map)
(:kafka/key-deserializer defaults))
"value.deserializer" (or (:kafka/deserializer task-map)
(:kafka/deserializer defaults))}
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)
consumer* (h/build-consumer consumer-config)
_ (when (and (:kafka/target-offsets task-map)
(:kafka/start-offsets task-map)
(not= (set (keys (:kafka/target-offsets task-map)))
Expand Down Expand Up @@ -394,14 +400,16 @@
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}
"max.request.size" request-size
"key.serializer" (or (:kafka/key-serializer task-map)
(:kafka/key-serializer defaults))
"value.serializer" (or (:kafka/serializer task-map)
(:kafka/serializer defaults))}
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)
producer (h/build-producer config)
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)
Expand Down
4 changes: 4 additions & 0 deletions src/onyx/tasks/kafka.clj
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
:kafka/deserializer-fn os/NamespacedKeyword
(s/optional-key :kafka/bootstrap-servers) [s/Str]
(s/optional-key :kafka/zookeeper) s/Str
(s/optional-key :kafka/deserializer) s/Str
(s/optional-key :kafka/key-deserializer) s/Str
(s/optional-key :kafka/key-deserializer-fn) os/NamespacedKeyword
(s/optional-key :kafka/group-id) s/Str
(s/optional-key :kafka/consumer-opts) {s/Any s/Any}
Expand Down Expand Up @@ -79,6 +81,8 @@
:kafka/serializer-fn os/NamespacedKeyword
(s/optional-key :kafka/bootstrap-servers) [s/Str]
(s/optional-key :kafka/zookeeper) s/Str
(s/optional-key :kafka/serializer) s/Str
(s/optional-key :kafka/key-serializer) s/Str
(s/optional-key :kafka/key-serializer-fn) os/NamespacedKeyword
(s/optional-key :kafka/request-size) s/Num
(s/optional-key :kafka/partition) (s/cond-pre s/Int s/Str)
Expand Down
8 changes: 4 additions & 4 deletions test/onyx/plugin/input_benchmark_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -74,10 +74,10 @@
(defn write-data
[topic zookeeper bootstrap-servers]
(h/create-topic! zookeeper topic n-partitions 1)
(let [producer-config {"bootstrap.servers" bootstrap-servers}
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)
producer1 (h/build-producer producer-config key-serializer value-serializer)]
(let [producer-config {"bootstrap.servers" bootstrap-servers
"key.serializer" (h/byte-array-serializer-name)
"value.serializer" (h/byte-array-serializer-name)}
producer1 (h/build-producer producer-config)]
(time
(doseq [p (range n-partitions)]
(mapv deref
Expand Down
10 changes: 5 additions & 5 deletions test/onyx/plugin/input_multipartition_scheduling_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -40,11 +40,11 @@
(defn write-data
[topic zookeeper kafka-bootstrap n-segments-each]
(h/create-topic! zookeeper topic n-partitions 1)
(let [producer-config {"bootstrap.servers" kafka-bootstrap}
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)]
(with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)]
(with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)]
(let [producer-config {"bootstrap.servers" kafka-bootstrap
"key.serializer" (h/byte-array-serializer-name)
"value.serializer" (h/byte-array-serializer-name)}]
(with-open [producer1 (h/build-producer producer-config)]
(with-open [producer2 (h/build-producer producer-config)]
(doseq [x (range n-segments-each)] ;0 1 2
(h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x}))))
(doseq [x (range n-segments-each)] ;3 4 5
Expand Down
10 changes: 5 additions & 5 deletions test/onyx/plugin/input_seek_latest_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -42,11 +42,11 @@
"Use a custom version of mock-kafka as opposed to the one in test-utils
because we need to spawn 2 producers in order to write to each partition"
[topic zookeeper bootstrap-servers]
(let [producer-config {"bootstrap.servers" bootstrap-servers}
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)]
(with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)]
(with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)]
(let [producer-config {"bootstrap.servers" bootstrap-servers
"key.serializer" (h/byte-array-serializer-name)
"value.serializer" (h/byte-array-serializer-name)}]
(with-open [producer1 (h/build-producer producer-config)]
(with-open [producer2 (h/build-producer producer-config)]
(doseq [x (range 3)] ;0 1 2
(h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x}))))
(doseq [x (range 3)] ;3 4 5
Expand Down
8 changes: 4 additions & 4 deletions test/onyx/plugin/input_start_offset_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,10 @@
(defn write-data-out
[topic zookeeper bootstrap-servers]
(h/create-topic! zookeeper topic 1 1)
(let [producer-config {"bootstrap.servers" bootstrap-servers}
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)]
(with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)]
(let [producer-config {"bootstrap.servers" bootstrap-servers
"key.serializer" (h/byte-array-serializer-name)
"value.serializer" (h/byte-array-serializer-name)}]
(with-open [producer1 (h/build-producer producer-config)]
(doseq [x (range 5)] ;0 1 2
(Thread/sleep 500)
(h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x})))))))
Expand Down
10 changes: 5 additions & 5 deletions test/onyx/plugin/input_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -41,11 +41,11 @@
(defn write-data
[topic zookeeper bootstrap-servers]
(h/create-topic! zookeeper topic n-partitions 1)
(let [producer-config {"bootstrap.servers" bootstrap-servers}
key-serializer (h/byte-array-serializer)
value-serializer (h/byte-array-serializer)]
(with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)]
(with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)]
(let [producer-config {"bootstrap.servers" bootstrap-servers
"key.serializer" (h/byte-array-serializer-name)
"value.serializer" (h/byte-array-serializer-name)}]
(with-open [producer1 (h/build-producer producer-config)]
(with-open [producer2 (h/build-producer producer-config)]
(doseq [x (range 3)] ;0 1 2
(h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x}))))
(doseq [x (range 3)] ;3 4 5
Expand Down
2 changes: 1 addition & 1 deletion test/onyx/plugin/output_bench_test.clj
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@
(log/info "Waiting on messages in" test-topic)
(let [run-time (- (System/currentTimeMillis) start-time)
_ (println (float (* 1000 (/ n-messages-total run-time))) "messages per second. Processed" n-messages-total "messages in" run-time "ms.")
;msgs (take-now zk-address test-topic decompress 180000)
;msgs (h/take-now zk-address test-topic decompress 180000 (h/byte-array-deserializer-name) (h/byte-array-deserializer-name))
]
;(is (= (map :n test-data) (map :value msgs)))
)))
Expand Down
Loading