├── .gitignore ├── LICENSE ├── README.md ├── dev └── logback.xml ├── project.clj ├── src └── kafka_streams_clojure │ └── api.clj └── test └── kafka_streams_clojure ├── api_test.clj └── embedded_kafka.clj /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /classes 3 | /checkouts 4 | pom.xml 5 | pom.xml.asc 6 | *.jar 7 | *.class 8 | /.lein-* 9 | /.nrepl-port 10 | /log 11 | .hgignore 12 | .hg/ 13 | 14 | /.idea 15 | /*.iml 16 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright 2017 Bobby Calderwood 2 | 3 | Licensed under the Apache License, Version 2.0 (the "License"); 4 | you may not use this file except in compliance with the License. 5 | You may obtain a copy of the License at 6 | 7 | http://www.apache.org/licenses/LICENSE-2.0 8 | 9 | Unless required by applicable law or agreed to in writing, software 10 | distributed under the License is distributed on an "AS IS" BASIS, 11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | See the License for the specific language governing permissions and 13 | limitations under the License. -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # kafka-streams-clojure 2 | 3 | [Clojure transducers](https://clojure.org/reference/transducers) 4 | interface to 5 | [Kafka Streams](https://kafka.apache.org/documentation/streams). This 6 | combo provides the best of both worlds for building streaming 7 | applications on Kafka with Clojure: 8 | 9 | * Simple, declarative, idiomatic, composable, testable stream 10 | transformation logic, via transducers 11 | * Easy, battle-hardened distributed system topology specification, 12 | cluster partition rebalancing, local state management, etc. via Kafka 13 | Streams 14 | 15 | ## Status 16 | 17 | **THIS LIBRARY IS CURRENTLY ALPHA STATUS, AND IS NOT FIT FOR PRODUCTION USE!** 18 | 19 | This notice will be removed when I believe the API is stable and the 20 | library has performed well under heavy loads in real-world use. 21 | 22 | ### Features & Roadmap 23 | 24 | Currently, this library supports: 25 | 26 | * Hooking a transducer into a `KStream` processing pipeline. 27 | 28 | In the future, I plan for this library to support: 29 | 30 | * Helper transducers for stateful computations like joins, windowed 31 | aggregates, etc. to mirror the functionality of the `KStream` API, 32 | but which can be composed with purely functional steps 33 | * An appropriate level of integration into both the low-level 34 | `Processor` API and the `KTable` APIs. 35 | 36 | ## Installation 37 | 38 | **Note: Due to its alpha status, this library is not configured for 39 | CI/CD, and no JARs have been pushed to a public repository. You'll 40 | have to install (as per instructions below) into your local Maven repo 41 | before the following instructions will work** 42 | 43 | Include the library JAR in your Boot/Leiningen dependencies: 44 | 45 | ``` clojure 46 | [kafka-streams-clojure "0.1.0-SNAPSHOT"] 47 | ``` 48 | 49 | ### Kafka Streams Dependency 50 | 51 | Kafka Streams is included as a `provided` dependency, meaning your 52 | application will need to include the 53 | [Kafka Streams JAR](https://mvnrepository.com/artifact/org.apache.kafka/kafka-streams) 54 | as a dependency as well as this library. 55 | 56 | ## Usage 57 | 58 | Transducers provide a more Clojure-idiomatic way to transform 59 | streaming key value pairs than `KStream`'s Java 8 Streams-like API. 60 | The key function is `kafka-streams-clojure.api/transduce-kstream`, 61 | which makes the given `KStream` a transducible context by applying the 62 | given transducer as a `Transformer`. The step function is invoked 63 | with the `ProcessorContext` and a 2-tuple of `[key value]` for each 64 | record, so the transducer should be shaped accordingly. 65 | 66 | This library also provides a number of stateful transducers over Kafka 67 | Streams' Stores API for doing joins, windowed aggregates, etc. The 68 | goal of this library is to maintain feature parity with the high-level 69 | `KStream`, `KTable`, etc. APIs, as well as (eventually) to enable 70 | transducer usage in the low-level `Processor` API. 71 | 72 | ``` clojure 73 | // Start Kafka Cluster running locally 74 | 75 | (require '[kafka-streams-clojure.api :as api]) 76 | (import '[org.apache.kafka.clients.producer KafkaProducer ProducerRecord] 77 | '[org.apache.kafka.streams StreamsConfig KafkaStreams] 78 | '[org.apache.kafka.streams.kstream KStreamBuilder]) 79 | 80 | (def xform (comp (filter (fn [[k v]] (string? v))) 81 | (map (fn [[k v]] [v k])) 82 | (filter (fn [[k v]] (= "foo" v))))) 83 | (def builder (KStreamBuilder.)) 84 | (def kstream (-> builder 85 | (.stream (into-array String ["tset"])) 86 | (api/transduce-kstream xform) 87 | (.to "test"))) 88 | 89 | (def kafka-streams 90 | (KafkaStreams. builder 91 | (StreamsConfig. {StreamsConfig/APPLICATION_ID_CONFIG "test-app-id" 92 | StreamsConfig/BOOTSTRAP_SERVERS_CONFIG "localhost:9092" 93 | StreamsConfig/KEY_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde 94 | StreamsConfig/VALUE_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde}))) 95 | (.start kafka-streams) 96 | 97 | (def producer (KafkaProducer. {"bootstrap.servers" "localhost:9092" 98 | "acks" "all" 99 | "retries" "0" 100 | "key.serializer" "org.apache.kafka.common.serialization.StringSerializer" 101 | "value.serializer" "org.apache.kafka.common.serialization.StringSerializer"})) 102 | 103 | @(.send producer (ProducerRecord. "tset" "foo" "bar")) 104 | // Observe message come across topic "test" via kafka-console-consumer 105 | 106 | @(.send producer (ProducerRecord. "tset" "baz" "quux")) 107 | // Observe message does not come across topic "test" via kafka-console-consumer 108 | 109 | (.close producer) 110 | (.close kafka-streams) 111 | ``` 112 | 113 | ## Dev, Build, Test 114 | 115 | This project uses [Leiningen](https://leiningen.org/) for dev, test, 116 | and build workflow. 117 | 118 | ### Run Tests 119 | 120 | The test include an embedded, single-node Kafka/ZooKeeper cluster that 121 | runs on demand. 122 | 123 | ``` bash 124 | lein test 125 | ``` 126 | 127 | ### Run REPL 128 | 129 | To run via the REPL, you'll need to fire up a Kafka Cluster. 130 | 131 | ``` bash 132 | lein repl 133 | ``` 134 | 135 | ### Build and Push JAR 136 | 137 | ``` bash 138 | lein jar 139 | lein deploy 140 | ``` 141 | 142 | ## License 143 | 144 | ``` 145 | Copyright 2017 Bobby Calderwood 146 | 147 | Licensed under the Apache License, Version 2.0 (the "License"); 148 | you may not use this file except in compliance with the License. 149 | You may obtain a copy of the License at 150 | 151 | http://www.apache.org/licenses/LICENSE-2.0 152 | 153 | Unless required by applicable law or agreed to in writing, software 154 | distributed under the License is distributed on an "AS IS" BASIS, 155 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 156 | See the License for the specific language governing permissions and 157 | limitations under the License. 158 | ``` 159 | -------------------------------------------------------------------------------- /dev/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | WARN 12 | 13 | 14 | 15 | 16 | 17 | %date{HH:mm:ss.SSS} %-5level %logger [%thread]%n%msg%n 18 | UTF-8 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | %date{HH:mm:ss.SSS} %-5level %logger{25}: %msg %X thread=%thread%n 29 | 30 | 31 | 32 | log/all.log 33 | 34 | 35 | 36 | log/all.%d{yyyy-MM-dd}.%i.log 37 | 38 | 39 | 64 MB 40 | 41 | 42 | 15 43 | 44 | 45 | 512MB 46 | 47 | 48 | true 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | %date{HH:mm:ss.SSS} %-5level %logger{25}: %msg %X thread=%thread%n 59 | 60 | 61 | 62 | log/app.log 63 | 64 | 65 | 66 | log/app.%d{yyyy-MM-dd}.%i.log 67 | 68 | 69 | 64 MB 70 | 71 | 72 | 15 73 | 74 | 75 | 512MB 76 | 77 | 78 | true 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 98 | 99 | 100 | 101 | 102 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 114 | 115 | true 116 | 117 | 118 | -------------------------------------------------------------------------------- /project.clj: -------------------------------------------------------------------------------- 1 | (def kafka-version "1.0.0") 2 | 3 | (defproject kafka-streams-clojure "0.1.0-SNAPSHOT" 4 | :description "Kafka Streams integration with Clojure transducers." 5 | :url "https://github.com/bobby/kafka-streams-clojure" 6 | :license {:name "Apache License, Version 2.0" 7 | :url "http://www.apache.org/licenses/LICENSE-2.0"} 8 | :dependencies [[org.clojure/clojure "1.8.0"] 9 | [org.apache.kafka/kafka-streams ~kafka-version :scope "provided"]] 10 | :profiles {:dev {:resource-paths ["dev"] 11 | :dependencies [[org.apache.kafka/kafka_2.11 ~kafka-version] 12 | [ch.qos.logback/logback-classic "1.1.7"] 13 | [org.slf4j/jcl-over-slf4j "1.7.21"] 14 | [org.slf4j/log4j-over-slf4j "1.7.21"] 15 | [org.apache.logging.log4j/log4j-to-slf4j "2.7"] 16 | [org.slf4j/jul-to-slf4j "1.7.21"]] 17 | :exclusions [commons-logging 18 | log4j 19 | org.apache.logging.log4j/log4j 20 | org.slf4j/simple 21 | org.slf4j/slf4j-jcl 22 | org.slf4j/slf4j-nop 23 | org.slf4j/slf4j-log4j12 24 | org.slf4j/slf4j-log4j13 25 | org.slf4j/osgi-over-slf4j]}}) 26 | -------------------------------------------------------------------------------- /src/kafka_streams_clojure/api.clj: -------------------------------------------------------------------------------- 1 | (ns kafka-streams-clojure.api 2 | (:refer-clojure :exclude [get range]) 3 | (:import [org.apache.kafka.streams.kstream 4 | Transformer 5 | TransformerSupplier 6 | KStream 7 | Predicate 8 | ValueJoiner 9 | KeyValueMapper] 10 | [org.apache.kafka.streams.processor ProcessorContext] 11 | [org.apache.kafka.streams StreamsBuilder KeyValue] 12 | [org.apache.kafka.streams.state KeyValueIterator ReadOnlyKeyValueStore])) 13 | 14 | (set! *warn-on-reflection* true) 15 | 16 | (deftype TransducerTransformer [step-fn ^{:volatile-mutable true} context] 17 | Transformer 18 | (init [_ c] 19 | (set! context c)) 20 | (transform [_ k v] 21 | (try 22 | (step-fn context [k v]) 23 | (catch Exception e 24 | (.printStackTrace e))) 25 | nil) 26 | (punctuate [^Transformer this ^long t]) 27 | (close [_])) 28 | 29 | (defn- kafka-streams-step 30 | ([context] context) 31 | ([^ProcessorContext context [k v]] 32 | (.forward context k v) 33 | (.commit context) 34 | context)) 35 | 36 | (defn transformer 37 | "Creates a transducing transformer for use in Kafka Streams topologies." 38 | [xform] 39 | (TransducerTransformer. (xform kafka-streams-step) nil)) 40 | 41 | (defn transformer-supplier 42 | [xform] 43 | (reify 44 | TransformerSupplier 45 | (get [_] (transformer xform)))) 46 | 47 | ;;; TODO: Wrap fluent build DSL in something more Clojure-y (macro?!?!?) 48 | 49 | (defn transduce-kstream 50 | ^KStream [^KStream kstream xform] 51 | (.transform kstream (transformer-supplier xform) (into-array String []))) 52 | 53 | ;;; TODO: Reproduce useful KStream, KTable APIs here as transducers 54 | ;;; (i.e. for those not already covered by existing map, filter, etc.) 55 | ;;; (e.g. leftJoin, through, etc.) 56 | 57 | (defn ^KStream stream 58 | "Clojure wrapper around StreamsBuilder.stream(String names...)" 59 | [^StreamsBuilder builder & stream-names] 60 | (.stream builder stream-names)) 61 | 62 | (defn branch 63 | "Clojure wrapper around KStream.branch(Predicate predicates...). 64 | Accepts a KStream instance and a variable number of arity-1 65 | predicates of [k v]." 66 | [^KStream kstream & predicates] 67 | (let [preds (into-array Predicate (map #(reify Predicate (test [_ k v] (% [k v]))) predicates))] 68 | (into [] (.branch kstream preds)))) 69 | 70 | (defn branch-map 71 | "Given a KStream instance and a map of 72 | 73 | `keyword-branch-name -> (arity-1 predicate of [k v])` 74 | 75 | returns a map of 76 | 77 | `keyword-branch-name -> KStream` 78 | 79 | as per KStream.branch" 80 | [^KStream kstream branch-predicate-map] 81 | (let [[branch-names predicates] (reduce (fn [agg [k v]] 82 | (-> agg 83 | (update-in [0] conj k) 84 | (update-in [1] conj v))) 85 | [[] []] 86 | branch-predicate-map) 87 | kstreams (apply branch kstream predicates)] 88 | (zipmap branch-names kstreams))) 89 | 90 | (defn ^ValueJoiner value-joiner 91 | [f] 92 | (reify ValueJoiner 93 | (apply [_ v1 v2] 94 | (f v1 v2)))) 95 | 96 | (defn ^KeyValueMapper key-value-mapper 97 | [f] 98 | (reify KeyValueMapper 99 | (apply [_ k v] 100 | (f [k v])))) 101 | 102 | (defprotocol IReadOnlyKeyValueStore 103 | (-get [this key] 104 | "Gets the value at the given key.") 105 | (-all [this] 106 | "Returns a Seqable of all `[key value]` pairs in this store. The 107 | Seqable is also Closeable, and must be `.close`d after use.") 108 | (-range [this start end] 109 | "Returns a Seqable of `[key value]` pairs between keys `start` and 110 | `end`. The Seqable is also Closeable, and must be `.close`d after 111 | use.")) 112 | 113 | (defn get 114 | "Gets the value at the given key." 115 | [this _key] 116 | (-get this _key)) 117 | 118 | (defn all 119 | "Returns a Seqable of all `[key value]` pairs in this store. The 120 | Seqable is also Closeable, and must be `.close`d after use." 121 | [this] 122 | (-all this)) 123 | 124 | (defn range 125 | "Returns a Seqable of `[key value]` pairs between keys `start` and 126 | `end`. The Seqable is also Closeable, and must be `.close`d after 127 | use." 128 | [this start end] 129 | (-range this start end)) 130 | 131 | (deftype KeyValueTupleIterator [^KeyValueIterator iter] 132 | KeyValueIterator 133 | (hasNext [_] (.hasNext iter)) 134 | (next [_] 135 | (let [^KeyValue kv (.next iter)] 136 | [(.key kv) (.value kv)])) 137 | (remove [_] (throw (UnsupportedOperationException. "Not supported"))) 138 | (close [_] (.close iter)) 139 | (peekNextKey [_] (.peekNextKey iter))) 140 | 141 | (extend-type ReadOnlyKeyValueStore 142 | IReadOnlyKeyValueStore 143 | (-get [this _key] 144 | (.get this _key)) 145 | (-all [this] 146 | (-> this 147 | .all 148 | ->KeyValueTupleIterator)) 149 | (-range [this start end] 150 | (-> this 151 | (.range start end) 152 | ->KeyValueTupleIterator))) 153 | 154 | (comment 155 | (import '[org.apache.kafka.streams StreamsConfig KafkaStreams StreamsBuilder]) 156 | 157 | (def xform (comp (filter (fn [[k v]] (string? v))) 158 | (map (fn [[k v]] [v k])) 159 | (filter (fn [[k v]] (= "foo" v))))) 160 | (def builder (StreamsBuilder.)) 161 | (def kstream (-> builder 162 | (stream "tset") 163 | (transduce-kstream xform) 164 | (.to "test"))) 165 | 166 | (def kafka-streams 167 | (KafkaStreams. (.build builder) 168 | (StreamsConfig. {StreamsConfig/APPLICATION_ID_CONFIG "test-app-id" 169 | StreamsConfig/BOOTSTRAP_SERVERS_CONFIG "localhost:9092" 170 | StreamsConfig/KEY_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde 171 | StreamsConfig/VALUE_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde}))) 172 | (.start kafka-streams) 173 | 174 | (import '[org.apache.kafka.clients.producer KafkaProducer ProducerRecord]) 175 | 176 | (def producer (KafkaProducer. {"bootstrap.servers" "localhost:9092" 177 | "acks" "all" 178 | "retries" "0" 179 | "key.serializer" "org.apache.kafka.common.serialization.StringSerializer" 180 | "value.serializer" "org.apache.kafka.common.serialization.StringSerializer"})) 181 | 182 | @(.send producer (ProducerRecord. "tset" "foo" "bar")) 183 | 184 | (.close producer) 185 | (.close kafka-streams) 186 | 187 | ) 188 | -------------------------------------------------------------------------------- /test/kafka_streams_clojure/api_test.clj: -------------------------------------------------------------------------------- 1 | (ns kafka-streams-clojure.api-test 2 | (:require [clojure.test :refer :all] 3 | [kafka-streams-clojure.embedded-kafka :refer [with-test-broker kafka-config]] 4 | [kafka-streams-clojure.api :as api]) 5 | (:import [kafka-streams-clojure.api ] 6 | [org.apache.kafka.clients.producer Producer ProducerRecord] 7 | [org.apache.kafka.clients.consumer Consumer ConsumerRecords ConsumerRecord] 8 | [org.apache.kafka.streams StreamsBuilder KafkaStreams StreamsConfig] 9 | [org.apache.kafka.streams.kstream KStream Transformer TransformerSupplier])) 10 | 11 | (set! *warn-on-reflection* true) 12 | 13 | (deftest test-kafka-streams-api-types 14 | (testing "Integration with Kafka Streams API types" 15 | (let [xform (comp (filter (fn [[k v]] (string? v))) 16 | (map (fn [[k v]] [v k])) 17 | (filter (fn [[k v]] (= "foo" v)))) 18 | supplier (api/transformer-supplier xform) 19 | transformer (api/transformer xform)] 20 | (is (instance? TransformerSupplier supplier)) 21 | #_(is (instance? ProcessorSupplier supplier)) 22 | 23 | (is (instance? Transformer transformer)) 24 | #_(is (instance? Processor transformer)) 25 | 26 | (is (instance? Transformer (.get supplier))) 27 | #_(is (instance? Processor (.get supplier)))))) 28 | 29 | (defn produce-records-synchronously 30 | [^Producer producer topic records] 31 | (doseq [[k v] records] 32 | @(.send producer (ProducerRecord. topic k v)))) 33 | 34 | (defn consume-n-records 35 | [^Consumer consumer topic n] 36 | (.subscribe consumer [topic]) 37 | (loop [records []] 38 | (if (< (count records) n) 39 | (recur (reduce (fn [agg ^ConsumerRecord record] (conj agg [(.key record) (.value record)])) 40 | records 41 | (.poll consumer 1000))) 42 | (take n records)))) 43 | 44 | (deftest test-integration 45 | (testing "Basic integration test" 46 | (with-test-broker producer consumer 47 | (let [input-topic "tset" 48 | output-topic "test" 49 | xform (comp (filter (fn [[k v]] (string? v))) 50 | (map (fn [[k v]] [v k])) 51 | (filter (fn [[k v]] (= "foo" v)))) 52 | builder (StreamsBuilder.) 53 | kstream (-> builder 54 | (api/stream input-topic) 55 | (api/transduce-kstream xform) 56 | (.to output-topic)) 57 | kafka-streams (KafkaStreams. (.build builder) 58 | (StreamsConfig. {StreamsConfig/APPLICATION_ID_CONFIG (str "test-app-id-" (java.util.UUID/randomUUID)) 59 | StreamsConfig/BOOTSTRAP_SERVERS_CONFIG (get kafka-config "bootstrap.servers") 60 | StreamsConfig/KEY_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde 61 | StreamsConfig/VALUE_SERDE_CLASS_CONFIG org.apache.kafka.common.serialization.Serdes$StringSerde})) 62 | input-values {"foo" "bar" 63 | "baz" "quux"}] 64 | (.start kafka-streams) 65 | 66 | (produce-records-synchronously producer input-topic input-values) 67 | (is (= [["bar" "foo"]] (consume-n-records consumer output-topic 1))) 68 | 69 | (.close kafka-streams))))) 70 | 71 | (deftest test-branch-and-stream 72 | (testing "stream takes a StreamsBuilder and returns a KStream" 73 | (let [parent-stream (api/stream (StreamsBuilder.) "tset")] 74 | (is (instance? KStream parent-stream)) 75 | (testing "branch takes a stream and returns" 76 | (doseq [kstream (api/branch parent-stream 77 | (fn [[k v]] (= k :foo)) 78 | (fn [[k v]] (= v :branch)))] 79 | (is (instance? KStream kstream))))))) 80 | 81 | (deftest test-branch-map 82 | (testing "stream takes a StreamsBuilder and returns a KStream" 83 | (let [parent-stream (api/stream (StreamsBuilder.) "tset")] 84 | (testing "branch takes a stream and returns" 85 | (doseq [[name kstream] (api/branch-map parent-stream 86 | {:foo (fn [[k v]] (= k :foo)) 87 | :branch (fn [[k v]] (= v :branch))})] 88 | (is (keyword? name)) 89 | (is (instance? KStream kstream))))))) 90 | -------------------------------------------------------------------------------- /test/kafka_streams_clojure/embedded_kafka.clj: -------------------------------------------------------------------------------- 1 | ;; from https://github.com/Mayvenn/embedded-kafka/blob/c0092b3e48a009238a645ee79d65e0f9de27999b/src/embedded_kafka/core.clj 2 | 3 | (ns kafka-streams-clojure.embedded-kafka 4 | (:import 5 | [kafka.server KafkaConfig KafkaServerStartable] 6 | [org.apache.kafka.clients.producer Producer KafkaProducer] 7 | [org.apache.kafka.clients.consumer Consumer KafkaConsumer] 8 | [java.net InetSocketAddress] 9 | [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxnFactory] 10 | [java.util Properties]) 11 | (:require [clojure.java.io :refer [file]])) 12 | 13 | (set! *warn-on-reflection* true) 14 | 15 | (defn safe-delete [file-path] 16 | (if (.exists (clojure.java.io/file file-path)) 17 | (try 18 | (clojure.java.io/delete-file file-path) 19 | (catch Exception e (str "exception: " (.getMessage e)))) 20 | false)) 21 | 22 | (defn delete-directory [directory-path] 23 | (let [directory-contents (file-seq (clojure.java.io/file directory-path)) 24 | files-to-delete (filter #(.isFile ^java.io.File %) directory-contents)] 25 | (doseq [^java.io.File file files-to-delete] 26 | (safe-delete (.getPath file))) 27 | (safe-delete directory-path))) 28 | 29 | (defn tmp-dir [& parts] 30 | (.getPath ^java.io.File (apply file (System/getProperty "java.io.tmpdir") "embedded-kafka" parts))) 31 | 32 | (def ^:dynamic kafka-config 33 | {"broker.id" "0" 34 | "listeners" "PLAINTEXT://localhost:9999" 35 | "bootstrap.servers" "localhost:9999" 36 | "zookeeper.connect" "127.0.0.1:2182" 37 | "zookeeper-port" "2182" 38 | "log.flush.interval.messages" "1" 39 | "auto.create.topics.enable" "true" 40 | "group.id" "consumer" 41 | "auto.offset.reset" "earliest" 42 | "retry.backoff.ms" "500" 43 | "message.send.max.retries" "5" 44 | "auto.commit.enable" "false" 45 | "offsets.topic.replication.factor" "1" 46 | "max.poll.records" "1" 47 | "log.dir" (.getAbsolutePath (file (tmp-dir "kafka-log"))) 48 | "acks" "all" 49 | "retries" "0" 50 | "key.serializer" "org.apache.kafka.common.serialization.StringSerializer" 51 | "value.serializer" "org.apache.kafka.common.serialization.StringSerializer" 52 | "key.deserializer" "org.apache.kafka.common.serialization.StringDeserializer" 53 | "value.deserializer" "org.apache.kafka.common.serialization.StringDeserializer"}) 54 | 55 | (defn as-properties [m] 56 | (let [ps (Properties.)] 57 | (doseq [[k v] m] (.setProperty ps k v)) 58 | ps)) 59 | 60 | (defn ^KafkaServerStartable create-broker [] 61 | (KafkaServerStartable. (KafkaConfig. (as-properties kafka-config)))) 62 | 63 | (defn ^NIOServerCnxnFactory create-zookeeper [] 64 | (let [tick-time 500 65 | zk (ZooKeeperServer. (file (tmp-dir "zookeeper-snapshot")) (file (tmp-dir "zookeeper-log")) tick-time)] 66 | (doto (NIOServerCnxnFactory.) 67 | (.configure (InetSocketAddress. (read-string (kafka-config "zookeeper-port"))) 60) 68 | (.startup zk)))) 69 | 70 | (defmacro with-test-broker 71 | "Creates an in-process broker that can be used to test against" 72 | [producer-name consumer-name & body] 73 | `(do (delete-directory (file (tmp-dir))) 74 | (let [zk# (create-zookeeper) 75 | kafka# (create-broker)] 76 | (try 77 | (.startup kafka#) 78 | (let [^Producer ~producer-name (KafkaProducer. ^java.util.Map kafka-config) 79 | ^Consumer ~consumer-name (KafkaConsumer. ^java.util.Map kafka-config)] 80 | (try 81 | ~@body 82 | (finally 83 | (do (.close ~consumer-name) 84 | (.close ~producer-name))))) 85 | (finally (do (.shutdown kafka#) 86 | (.awaitShutdown kafka#) 87 | (.shutdown zk#) 88 | (delete-directory (file (tmp-dir))))))))) 89 | --------------------------------------------------------------------------------