├── .gitignore ├── project.clj ├── CHANGELOG.md ├── README.md ├── test └── gregor │ └── core_test.clj ├── LICENSE └── src └── gregor └── core.clj /.gitignore: -------------------------------------------------------------------------------- 1 | codox 2 | doc/ 3 | /classes 4 | /checkouts 5 | pom.xml 6 | pom.xml.asc 7 | *.jar 8 | *.class 9 | /.lein-* 10 | /.nrepl-port 11 | .hgignore 12 | .hg/ 13 | target/ 14 | -------------------------------------------------------------------------------- /project.clj: -------------------------------------------------------------------------------- 1 | (defproject io.weft/gregor "1.0.1-SNAPSHOT" 2 | :min-lein-version "2.0.0" 3 | :description "Lightweight Clojure bindings for Kafka" 4 | :url "https://github.com/ccann/gregor.git" 5 | :license {:name "Eclipse Public License" 6 | :url "http://www.eclipse.org/legal/epl-v10.html"} 7 | :dependencies [[org.clojure/clojure "1.10.0"] 8 | [org.apache.kafka/kafka_2.12 "2.1.1"]] 9 | :plugins [[lein-eftest "0.5.6"]] 10 | :deploy-repositories {"clojars" {:url "https://clojars.org/repo" 11 | :sign-releases false 12 | :username :env 13 | :passphrase :env}} 14 | :release-tasks [["vcs" "assert-committed"] 15 | ["change" "version" "leiningen.release/bump-version" "release"] 16 | ["uberjar"] 17 | ["vcs" "commit"] 18 | ["vcs" "tag" "v" "--no-sign"] 19 | ["deploy" "clojars"] 20 | ["change" "version" "leiningen.release/bump-version"] 21 | ["vcs" "commit"] 22 | ["vcs" "push" "--no-verify"]]) 23 | -------------------------------------------------------------------------------- /CHANGELOG.md: -------------------------------------------------------------------------------- 1 | # Change Log 2 | All notable changes to Gregor will be documented in this file. 3 | 4 | ## [1.0.0] - 2019-02-26 5 | 6 | ### Changes 7 | - upgrade to Kafka 2.1.1 8 | - upgrade to Clojure 1.10 9 | - upgrade lein-eftest to 0.5.6 10 | - drop lein-codox for cljdoc 11 | - [PR #5](https://github.com/ccann/gregor/pull/5): close with timeout for `KafkaConsumer` 12 | - refactor docstrings 13 | 14 | ## [0.8.0] - 2018-11-7 15 | 16 | ### Changes 17 | - upgrade to Kafka `2.0.0` 18 | - support producer, consumer timestamp fields adding arities to send functions 19 | - avoid NPE in send-then 20 | 21 | ### Changes 22 | - upgrade to Kafka `0.11.0.0` 23 | 24 | ## [0.6.0] - 2017-07-14 25 | 26 | ### Changes 27 | - upgrade to Kafka `0.11.0.0` 28 | 29 | ## [0.5.1] - 2016-08-28 30 | 31 | ### Fixed 32 | - Fix rack-aware-mode-constant by replacing class with object instance. 33 | 34 | ## [0.5.0] - 2016-08-23 35 | 36 | ### Added 37 | - new function `topics` which lists existing Kafka topics. 38 | 39 | ### Changes 40 | - upgrade to Kafka `0.10.0.1` 41 | - `kafka.admin.AdminUtils/createTopic` acquired another argument in `0.10`. It's an 42 | additional optional argument for `create-topic`. 43 | 44 | ### Fixed 45 | - fix producer doc link 46 | - remove unused Java class import 47 | 48 | ## [0.4.1] - 2016-07-10 49 | 50 | ### Added 51 | - Closeable protocol for consumers and producers via 52 | [Pull Request #9](https://github.com/weftio/gregor/pull/9) which includes closing a 53 | producer with a timeout in seconds. 54 | 55 | ### Changes 56 | - Use Kafka 0.9 compiled with Scala 2.11 instead of 2.10 because the Kafka maintainers 57 | recommend to do so. 58 | 59 | ## [0.4.0] - 2016-06-08 60 | 61 | ### Added 62 | Topic management via [Pull Request #5](https://github.com/weftio/gregor/pull/5): 63 | 64 | - `create-topic`: Create a new topic. 65 | - `topic-exists?`: Check if a topic exists. 66 | - `delete-topic`: Delete an existing topic. 67 | 68 | ### Fixed 69 | - [Pull Request #7](https://github.com/weftio/gregor/pull/7): Avoid a 70 | `NullPointerException` in the rebalance listener. 71 | - [Pull Request #4](https://github.com/weftio/gregor/pull/4): Exclude 72 | `clojure.core/flush` and `clojure.core/send`. 73 | 74 | ## [0.3.1] - 2016-05-18 75 | 76 | ### Added 77 | - vars for byte-array (de)serializers 78 | 79 | ## [0.3.0] - 2016-05-09 80 | 81 | Apologies for the several (pretty minor) breaking changes. 82 | 83 | ### Breaking Changes 84 | - Arity of `resume` and `pause` now aligns with the rest of the API (`assoc`-like 85 | optional arg pairs) 86 | - `commit-offsets-async!` and `commit-offsets!` optional arg `offsets` is now a seq of 87 | maps with `:topic`, `:partition`, `:offset` and optional `:metadata` keys. 88 | - `commited` now returns `nil` or a map with `:offset` and `:metadata` keys. 89 | - `send` no longer supports a callback, use `send-then` instead. 90 | 91 | ### Changes 92 | - Second `seek-to!` arg is now named `offset`. 93 | - `send` has new arities that correspond to those of the `ProducerRecord` constructor. 94 | 95 | ### Fixed 96 | - `resume` and `pause` no longer have the same implementation. 97 | - Merge pull request from `lambdahands` which fixes issue w/ overwritten custom 98 | (de)serializers in producer and consumer configs. 99 | 100 | ### Added 101 | - `send-then` function which provides `send` a callback. This callback expects a map of 102 | metadata and an exception as its args. 103 | - `->producer-record` function. 104 | 105 | 106 | ## [0.2.0] - 2016-03-25 107 | 108 | ### Added 109 | - First public release: added most of API 110 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Gregor 2 | 3 | Lightweight Clojure wrapper for [Apache Kafka](http://kafka.apache.org/) Consumer + Producer APIs. 4 | 5 | ```clojure 6 | :dependencies [[org.clojure/clojure "1.10.0"] 7 | [org.apache.kafka/kafka_2.12 "2.1.1"]] 8 | 9 | ``` 10 | [![cljdoc Badge](https://cljdoc.org/badge/io.weft/gregor)](https://cljdoc.org/d/io.weft/gregor/CURRENT) 11 | 12 | [![Clojars Project](https://clojars.org/io.weft/gregor/latest-version.svg)](https://clojars.org/io.weft/gregor) 13 | 14 | [**CHANGELOG**](https://github.com/ccann/gregor/blob/master/CHANGELOG.md) 15 | 16 | Gregor wraps most of the Java API for the Kafka [Producer](http://kafka.apache.org/0100/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html) and [New Consumer](http://kafka.apache.org/0100/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html) The goal of this project is to stay very close to the Kafka API instead of adding more advanced features. 17 | 18 | ## Example 19 | 20 | Here's an example of at-least-once processing (using [`mount`](https://github.com/tolitius/mount)): 21 | 22 | ```clojure 23 | (ns gregor-sample-app.core 24 | (:gen-class) 25 | (:require [clojure.repl :as repl] 26 | [gregor.core :as gregor] 27 | [mount.core :as mount :refer [defstate]])) 28 | 29 | (def run (atom true)) 30 | 31 | (defstate consumer 32 | :start (gregor/consumer "localhost:9092" 33 | "testgroup" 34 | ["test-topic"] 35 | {"auto.offset.reset" "earliest" 36 | "enable.auto.commit" "false"}) 37 | :stop (gregor/close consumer)) 38 | 39 | (defstate producer 40 | :start (gregor/producer "localhost:9092") 41 | :stop (gregor/close producer)) 42 | 43 | (defn -main 44 | [& args] 45 | (mount/start) 46 | (repl/set-break-handler! (fn [sig] (reset! run false))) 47 | (while @run 48 | (let [consumer-records (gregor/poll consumer) 49 | values (process-records consumer-records)] 50 | (doseq [v values] 51 | (gregor/send producer "other-topic" v)) 52 | (gregor/commit-offsets! consumer))) 53 | (mount/stop)) 54 | ``` 55 | 56 | Transformations over consumer records are applied in `process-records`. Each record in 57 | the `seq` returned by `poll` is a map. Here's an example with a JSON object as the 58 | `:value`: 59 | 60 | ```clojure 61 | {:value "{\"foo\":42}" 62 | :key nil 63 | :partition 0 64 | :topic "test-topic" 65 | :offset 939} 66 | ``` 67 | 68 | ## Producing 69 | 70 | Gregor provides the `send` function for asynchronously sending a record to a topic. There 71 | are multiple arities which correspond to those of the `ProducerRecord` 72 | [Java constructor](https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/producer/ProducerRecord.html). If 73 | you'd like to provide a callback to be invoked when the send has been acknowledged use 74 | `send-then` instead. 75 | 76 | ## Topic Management 77 | 78 | Create a topic: 79 | 80 | ```clojure 81 | (create-topic {:connection-string "localhost:2181"} "some-topic" {}) 82 | ``` 83 | That empty map can be used to specify configuration for number of topic partitions, replication factor, 84 | 85 | Delete a topic: 86 | 87 | ``` clojure 88 | (delete-topic {:connection-string "localhost:2181"} "some-topic") 89 | ``` 90 | 91 | Query about a topic's existence: 92 | 93 | ``` clojure 94 | (topic-exists? {:connection-string "localhost:2181"} "some-topic") 95 | ``` 96 | 97 | List existing topics: 98 | 99 | ``` clojure 100 | (topics {:connection-string "localhost:2181"}) 101 | ``` 102 | 103 | 104 | ## License 105 | 106 | Distributed under the Eclipse Public License either version 1.0 or (at your option) any 107 | later version. 108 | -------------------------------------------------------------------------------- /test/gregor/core_test.clj: -------------------------------------------------------------------------------- 1 | (ns gregor.core-test 2 | (:require [clojure.test :refer [deftest is]] 3 | [gregor.core :refer :all]) 4 | (:import java.util.concurrent.TimeUnit 5 | [org.apache.kafka.clients.consumer ConsumerRecord MockConsumer OffsetResetStrategy] 6 | org.apache.kafka.clients.producer.MockProducer 7 | org.apache.kafka.common.serialization.StringSerializer)) 8 | 9 | (deftest producing 10 | (let [p (MockProducer. true (StringSerializer.) (StringSerializer.))] 11 | (send p "unittest" {:a 1 :b "foo"}) 12 | (send-then p "unittest" {:a 2 :b "bar"} (fn [metadata ex])) 13 | (let [values (.history p) 14 | one (-> values first .value) 15 | two (-> values second .value)] 16 | (is (= {:a 1 :b "foo"} one)) 17 | (is (= {:a 2 :b "bar"} two)) 18 | (.close p)))) 19 | 20 | (deftest send-arities 21 | (let [time (System/currentTimeMillis) 22 | partition 0 23 | topic "unittest" 24 | p (MockProducer. true (StringSerializer.) (StringSerializer.)) 25 | key 0] 26 | 27 | (send p topic 1) 28 | (send-then p topic 2 (fn [_ _])) 29 | (is (= 1 (-> p .history first .value))) 30 | (is (= 2 (-> p .history second .value))) 31 | 32 | (send p topic key 3) 33 | (send-then p topic key 4 (fn [_ _])) 34 | (is (= 3 (-> p .history (#(nth % 2)) .value))) 35 | (is (= 4 (-> p .history (#(nth % 3)) .value))) 36 | 37 | (send p topic partition key 5) 38 | (send-then p topic partition key 6 (fn [_ _])) 39 | (is (= 5 (-> p .history (#(nth % 4)) .value))) 40 | (is (= 6 (-> p .history (#(nth % 5)) .value))) 41 | 42 | (send p topic partition time key 7) 43 | (send-then p topic partition time key 8 (fn [_ _])) 44 | (is (= 7 (-> p .history (#(nth % 6)) .value))) 45 | (is (= 8 (-> p .history (#(nth % 7)) .value))) 46 | 47 | (.close p))) 48 | 49 | (deftest subscribing 50 | (let [c (consumer "localhost:9092" "unit-test" ["test-topic"])] 51 | (is (= #{"test-topic"} (subscription c))) 52 | (unsubscribe c) 53 | (is (= #{} (subscription c))) 54 | (close c))) 55 | 56 | (deftest consuming 57 | (let [c (MockConsumer. (OffsetResetStrategy/EARLIEST)) 58 | _ (assign! c "test-topic" 0) 59 | c (doto c 60 | (.updateBeginningOffsets {(topic-partition "test-topic" 0) 0}) 61 | (.addRecord (ConsumerRecord. "test-topic" 0 0 0 {:a 1})) 62 | (.addRecord (ConsumerRecord. "test-topic" 0 1 0 {:b 2})) 63 | ;; duplicate offset (uniquely identifies each record within the partition) 64 | (.addRecord (ConsumerRecord. "test-topic" 0 0 0 {:c 3})) 65 | (.addRecord (ConsumerRecord. "test-topic" 0 2 0 {:d 4}))) 66 | ms (records c)] 67 | (is (= [{:a 1} {:b 2} {:d 4}] 68 | (mapv :value (first ms)))) 69 | (is (= #{(topic-partition "test-topic" 0)} 70 | (assignment c))) 71 | (.close c))) 72 | 73 | (deftest commit 74 | (let [c (doto (MockConsumer. (OffsetResetStrategy/EARLIEST)) 75 | (assign! "unittest" 0) 76 | (.updateBeginningOffsets {(topic-partition "unittest" 0) 0}) 77 | (.addRecord (ConsumerRecord. "unittest" 0 1 0 {:key :a})))] 78 | (is (= nil (committed c "unittest" 0))) 79 | (poll c) 80 | (commit-offsets! c) 81 | (is (= {:offset 2 :metadata nil} (committed c "unittest" 0))) 82 | (.addRecord c (ConsumerRecord. "unittest" 0 2 0 {:key :b})) 83 | (poll c) 84 | (commit-offsets-async! c) 85 | (is (= {:offset 3 :metadata nil} (committed c "unittest" 0))) 86 | (.addRecord c (ConsumerRecord. "unittest" 0 3 0 {:key :c})) 87 | (poll c) 88 | (commit-offsets-async! c (fn [om ex])) 89 | (is (= {:offset 4 :metadata nil} (committed c "unittest" 0))) 90 | (.addRecord c (ConsumerRecord. "unittest" 0 4 0 {:key :c})) 91 | (poll c) 92 | (is (= 5 (position c "unittest" 0))) 93 | (commit-offsets-async! c [{:topic "unittest" :partition 0 :offset 5}] (fn [om ex])) 94 | (is (= {:offset 5 :metadata nil} (committed c "unittest" 0))) 95 | (seek! c "unittest" 0 2) 96 | (is (= 2 (position c "unittest" 0))) 97 | (seek-to! c :beginning "unittest" 0) 98 | (is (= 0 (position c "unittest" 0))) 99 | (is (thrown? IllegalStateException 100 | (seek-to! c :end "unittest" 0) 101 | (= 5 (position c "unittest" 0)))))) 102 | 103 | (defn- zookeeper-config 104 | [] 105 | (when-let [connection-string (System/getenv "GREGOR_TEST_ZOOKEEPER")] 106 | {:connection-string connection-string 107 | :session-timeout 10000 108 | :connect-timeout 10000})) 109 | 110 | (defn- wait-for 111 | [pred timeout-ms] 112 | (let [start (System/currentTimeMillis)] 113 | (while (and 114 | (not (pred)) 115 | (< (System/currentTimeMillis) (+ start timeout-ms))) 116 | (Thread/sleep 100)))) 117 | 118 | (deftest topic-management 119 | (when-let [zk-config (zookeeper-config)] 120 | (let [topic "test-topic"] 121 | (is (not (topic-exists? zk-config topic))) 122 | 123 | (create-topic zk-config topic {}) 124 | (wait-for #(topic-exists? zk-config topic) 10000) 125 | (is (topic-exists? zk-config topic)) 126 | 127 | (is (not (empty? (topics zk-config)))) 128 | (is (some #{topic} (topics zk-config))) 129 | 130 | (delete-topic zk-config topic) 131 | (wait-for #(not (topic-exists? zk-config topic)) 10000) 132 | (is (not (topic-exists? zk-config topic)))))) 133 | 134 | (extend-protocol Closeable 135 | MockProducer 136 | (close ([p] (.close p)) 137 | ([p timeout] (.close p timeout TimeUnit/SECONDS))) 138 | MockConsumer 139 | (close ([c] (.close c)))) 140 | 141 | (deftest closing 142 | (let [p1 (MockProducer. true (StringSerializer.) (StringSerializer.)) 143 | p2 (MockProducer. true (StringSerializer.) (StringSerializer.)) 144 | c (MockConsumer. (OffsetResetStrategy/EARLIEST))] 145 | ;; mocks do not throw on send/poll when they are closed, so sanity check: 146 | (close p1) 147 | (close p2 42) 148 | (close c))) 149 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC 2 | LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM 3 | CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. 4 | 5 | 1. DEFINITIONS 6 | 7 | "Contribution" means: 8 | 9 | a) in the case of the initial Contributor, the initial code and 10 | documentation distributed under this Agreement, and 11 | 12 | b) in the case of each subsequent Contributor: 13 | 14 | i) changes to the Program, and 15 | 16 | ii) additions to the Program; 17 | 18 | where such changes and/or additions to the Program originate from and are 19 | distributed by that particular Contributor. A Contribution 'originates' from 20 | a Contributor if it was added to the Program by such Contributor itself or 21 | anyone acting on such Contributor's behalf. Contributions do not include 22 | additions to the Program which: (i) are separate modules of software 23 | distributed in conjunction with the Program under their own license 24 | agreement, and (ii) are not derivative works of the Program. 25 | 26 | "Contributor" means any person or entity that distributes the Program. 27 | 28 | "Licensed Patents" mean patent claims licensable by a Contributor which are 29 | necessarily infringed by the use or sale of its Contribution alone or when 30 | combined with the Program. 31 | 32 | "Program" means the Contributions distributed in accordance with this 33 | Agreement. 34 | 35 | "Recipient" means anyone who receives the Program under this Agreement, 36 | including all Contributors. 37 | 38 | 2. GRANT OF RIGHTS 39 | 40 | a) Subject to the terms of this Agreement, each Contributor hereby grants 41 | Recipient a non-exclusive, worldwide, royalty-free copyright license to 42 | reproduce, prepare derivative works of, publicly display, publicly perform, 43 | distribute and sublicense the Contribution of such Contributor, if any, and 44 | such derivative works, in source code and object code form. 45 | 46 | b) Subject to the terms of this Agreement, each Contributor hereby grants 47 | Recipient a non-exclusive, worldwide, royalty-free patent license under 48 | Licensed Patents to make, use, sell, offer to sell, import and otherwise 49 | transfer the Contribution of such Contributor, if any, in source code and 50 | object code form. This patent license shall apply to the combination of the 51 | Contribution and the Program if, at the time the Contribution is added by the 52 | Contributor, such addition of the Contribution causes such combination to be 53 | covered by the Licensed Patents. The patent license shall not apply to any 54 | other combinations which include the Contribution. No hardware per se is 55 | licensed hereunder. 56 | 57 | c) Recipient understands that although each Contributor grants the licenses 58 | to its Contributions set forth herein, no assurances are provided by any 59 | Contributor that the Program does not infringe the patent or other 60 | intellectual property rights of any other entity. Each Contributor disclaims 61 | any liability to Recipient for claims brought by any other entity based on 62 | infringement of intellectual property rights or otherwise. As a condition to 63 | exercising the rights and licenses granted hereunder, each Recipient hereby 64 | assumes sole responsibility to secure any other intellectual property rights 65 | needed, if any. For example, if a third party patent license is required to 66 | allow Recipient to distribute the Program, it is Recipient's responsibility 67 | to acquire that license before distributing the Program. 68 | 69 | d) Each Contributor represents that to its knowledge it has sufficient 70 | copyright rights in its Contribution, if any, to grant the copyright license 71 | set forth in this Agreement. 72 | 73 | 3. REQUIREMENTS 74 | 75 | A Contributor may choose to distribute the Program in object code form under 76 | its own license agreement, provided that: 77 | 78 | a) it complies with the terms and conditions of this Agreement; and 79 | 80 | b) its license agreement: 81 | 82 | i) effectively disclaims on behalf of all Contributors all warranties and 83 | conditions, express and implied, including warranties or conditions of title 84 | and non-infringement, and implied warranties or conditions of merchantability 85 | and fitness for a particular purpose; 86 | 87 | ii) effectively excludes on behalf of all Contributors all liability for 88 | damages, including direct, indirect, special, incidental and consequential 89 | damages, such as lost profits; 90 | 91 | iii) states that any provisions which differ from this Agreement are offered 92 | by that Contributor alone and not by any other party; and 93 | 94 | iv) states that source code for the Program is available from such 95 | Contributor, and informs licensees how to obtain it in a reasonable manner on 96 | or through a medium customarily used for software exchange. 97 | 98 | When the Program is made available in source code form: 99 | 100 | a) it must be made available under this Agreement; and 101 | 102 | b) a copy of this Agreement must be included with each copy of the Program. 103 | 104 | Contributors may not remove or alter any copyright notices contained within 105 | the Program. 106 | 107 | Each Contributor must identify itself as the originator of its Contribution, 108 | if any, in a manner that reasonably allows subsequent Recipients to identify 109 | the originator of the Contribution. 110 | 111 | 4. COMMERCIAL DISTRIBUTION 112 | 113 | Commercial distributors of software may accept certain responsibilities with 114 | respect to end users, business partners and the like. While this license is 115 | intended to facilitate the commercial use of the Program, the Contributor who 116 | includes the Program in a commercial product offering should do so in a 117 | manner which does not create potential liability for other Contributors. 118 | Therefore, if a Contributor includes the Program in a commercial product 119 | offering, such Contributor ("Commercial Contributor") hereby agrees to defend 120 | and indemnify every other Contributor ("Indemnified Contributor") against any 121 | losses, damages and costs (collectively "Losses") arising from claims, 122 | lawsuits and other legal actions brought by a third party against the 123 | Indemnified Contributor to the extent caused by the acts or omissions of such 124 | Commercial Contributor in connection with its distribution of the Program in 125 | a commercial product offering. The obligations in this section do not apply 126 | to any claims or Losses relating to any actual or alleged intellectual 127 | property infringement. In order to qualify, an Indemnified Contributor must: 128 | a) promptly notify the Commercial Contributor in writing of such claim, and 129 | b) allow the Commercial Contributor tocontrol, and cooperate with the 130 | Commercial Contributor in, the defense and any related settlement 131 | negotiations. The Indemnified Contributor may participate in any such claim 132 | at its own expense. 133 | 134 | For example, a Contributor might include the Program in a commercial product 135 | offering, Product X. That Contributor is then a Commercial Contributor. If 136 | that Commercial Contributor then makes performance claims, or offers 137 | warranties related to Product X, those performance claims and warranties are 138 | such Commercial Contributor's responsibility alone. Under this section, the 139 | Commercial Contributor would have to defend claims against the other 140 | Contributors related to those performance claims and warranties, and if a 141 | court requires any other Contributor to pay any damages as a result, the 142 | Commercial Contributor must pay those damages. 143 | 144 | 5. NO WARRANTY 145 | 146 | EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON 147 | AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER 148 | EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR 149 | CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A 150 | PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the 151 | appropriateness of using and distributing the Program and assumes all risks 152 | associated with its exercise of rights under this Agreement , including but 153 | not limited to the risks and costs of program errors, compliance with 154 | applicable laws, damage to or loss of data, programs or equipment, and 155 | unavailability or interruption of operations. 156 | 157 | 6. DISCLAIMER OF LIABILITY 158 | 159 | EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY 160 | CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, 161 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION 162 | LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 163 | CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 164 | ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE 165 | EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY 166 | OF SUCH DAMAGES. 167 | 168 | 7. GENERAL 169 | 170 | If any provision of this Agreement is invalid or unenforceable under 171 | applicable law, it shall not affect the validity or enforceability of the 172 | remainder of the terms of this Agreement, and without further action by the 173 | parties hereto, such provision shall be reformed to the minimum extent 174 | necessary to make such provision valid and enforceable. 175 | 176 | If Recipient institutes patent litigation against any entity (including a 177 | cross-claim or counterclaim in a lawsuit) alleging that the Program itself 178 | (excluding combinations of the Program with other software or hardware) 179 | infringes such Recipient's patent(s), then such Recipient's rights granted 180 | under Section 2(b) shall terminate as of the date such litigation is filed. 181 | 182 | All Recipient's rights under this Agreement shall terminate if it fails to 183 | comply with any of the material terms or conditions of this Agreement and 184 | does not cure such failure in a reasonable period of time after becoming 185 | aware of such noncompliance. If all Recipient's rights under this Agreement 186 | terminate, Recipient agrees to cease use and distribution of the Program as 187 | soon as reasonably practicable. However, Recipient's obligations under this 188 | Agreement and any licenses granted by Recipient relating to the Program shall 189 | continue and survive. 190 | 191 | Everyone is permitted to copy and distribute copies of this Agreement, but in 192 | order to avoid inconsistency the Agreement is copyrighted and may only be 193 | modified in the following manner. The Agreement Steward reserves the right to 194 | publish new versions (including revisions) of this Agreement from time to 195 | time. No one other than the Agreement Steward has the right to modify this 196 | Agreement. The Eclipse Foundation is the initial Agreement Steward. The 197 | Eclipse Foundation may assign the responsibility to serve as the Agreement 198 | Steward to a suitable separate entity. Each new version of the Agreement will 199 | be given a distinguishing version number. The Program (including 200 | Contributions) may always be distributed subject to the version of the 201 | Agreement under which it was received. In addition, after a new version of 202 | the Agreement is published, Contributor may elect to distribute the Program 203 | (including its Contributions) under the new version. Except as expressly 204 | stated in Sections 2(a) and 2(b) above, Recipient receives no rights or 205 | licenses to the intellectual property of any Contributor under this 206 | Agreement, whether expressly, by implication, estoppel or otherwise. All 207 | rights in the Program not expressly granted under this Agreement are 208 | reserved. 209 | 210 | This Agreement is governed by the laws of the State of New York and the 211 | intellectual property laws of the United States of America. No party to this 212 | Agreement will bring a legal action under this Agreement more than one year 213 | after the cause of action arose. Each party waives its rights to a jury trial 214 | in any resulting litigation. -------------------------------------------------------------------------------- /src/gregor/core.clj: -------------------------------------------------------------------------------- 1 | (ns gregor.core 2 | (:refer-clojure :exclude [flush send]) 3 | (:import [org.apache.kafka.common TopicPartition] 4 | [org.apache.kafka.clients.consumer Consumer KafkaConsumer ConsumerRecords 5 | ConsumerRecord OffsetAndMetadata OffsetCommitCallback 6 | ConsumerRebalanceListener] 7 | [org.apache.kafka.clients.producer Producer KafkaProducer Callback 8 | ProducerRecord RecordMetadata] 9 | [kafka.admin AdminUtils] 10 | [kafka.utils ZkUtils] 11 | [java.util.concurrent TimeUnit] 12 | [scala.collection JavaConversions]) 13 | (:require [clojure.set :as set] 14 | [clojure.string :as str])) 15 | 16 | (def ^:no-doc str-deserializer "org.apache.kafka.common.serialization.StringDeserializer") 17 | (def ^:no-doc str-serializer "org.apache.kafka.common.serialization.StringSerializer") 18 | (def ^:no-doc byte-array-deserializer "org.apache.kafka.common.serialization.ByteArrayDeserializer") 19 | (def ^:no-doc byte-array-serializer "org.apache.kafka.common.serialization.ByteArraySerializer") 20 | 21 | (defn- as-properties 22 | [m] 23 | (let [ps (java.util.Properties.)] 24 | (doseq [[k v] m] (.setProperty ps k v)) 25 | ps)) 26 | 27 | 28 | (defn topic-partition 29 | "A topic name and partition number." 30 | ^TopicPartition 31 | [^String topic ^Integer partition] 32 | (TopicPartition. topic partition)) 33 | 34 | 35 | (defn- arg-pairs 36 | [fn-name p1 p2 pairs] 37 | (let [pairs (remove nil? pairs)] 38 | (if (even? (count pairs)) 39 | (->> pairs 40 | (concat [p1 p2]) 41 | (partition 2)) 42 | (throw (IllegalArgumentException. 43 | (str fn-name 44 | " expects even number of optional args, found odd number.")))))) 45 | 46 | 47 | (defn- ->tps 48 | [fn-name topic partition tps] 49 | (let [pairs (arg-pairs fn-name topic partition tps)] 50 | (->> pairs (map #(apply topic-partition %))))) 51 | 52 | 53 | (defn- reify-occ 54 | [cb] 55 | (reify OffsetCommitCallback 56 | (onComplete [this offsets-map ex] 57 | (cb offsets-map ex)))) 58 | 59 | 60 | (defn- reify-crl 61 | [assigned-cb revoked-cb] 62 | (reify ConsumerRebalanceListener 63 | (onPartitionsAssigned [this partitions] 64 | (when assigned-cb 65 | (assigned-cb partitions))) 66 | (onPartitionsRevoked [this partitions] 67 | (when revoked-cb 68 | (revoked-cb partitions))))) 69 | 70 | 71 | (defn offset-and-metadata 72 | "Metadata for when an offset is committed." 73 | ([^Long offset] (OffsetAndMetadata. offset)) 74 | ([^Long offset ^String metadata] (OffsetAndMetadata. offset metadata))) 75 | 76 | 77 | (defn consumer-record->map 78 | [^ConsumerRecord record] 79 | {:value (.value record) 80 | :key (.key record) 81 | :partition (.partition record) 82 | :topic (.topic record) 83 | :offset (.offset record) 84 | :timestamp (.timestamp record) 85 | :timestamp-type (.toString (.timestampType record))}) 86 | 87 | 88 | (defprotocol Closeable 89 | "Provides two ways to close things: a default one with `(.close thing)` and the one 90 | with the specified timeout." 91 | (close [this] 92 | [this timeout])) 93 | 94 | 95 | (extend-protocol Closeable 96 | KafkaProducer 97 | (close 98 | ([p] (.close p)) 99 | ([p timeout] 100 | ;; Tries to close the producer cleanly within the specified timeout. 101 | ;; If the close does not complete within the timeout, fail any pending send 102 | ;; requests and force close the producer 103 | (.close p timeout TimeUnit/SECONDS))) 104 | KafkaConsumer 105 | (close 106 | ([c] (.close c)) 107 | ([p timeout] 108 | ;; Tries to close the consumer cleanly within the specified timeout. 109 | ;; If the consumer is unable to complete offset commits and gracefully leave 110 | ;; the group before the timeout expires, the consumer is force closed. 111 | (.close p timeout TimeUnit/SECONDS)))) 112 | 113 | 114 | ;;;;;;;;;;;;;;;;;;;; 115 | ;; Kafka Consumer ;; 116 | ;;;;;;;;;;;;;;;;;;;; 117 | 118 | 119 | (defn assign! 120 | "Manually assign topics and partitions to this consumer." 121 | [^Consumer consumer ^String topic ^Integer partition & tps] 122 | (->> tps 123 | (->tps "assign!" topic partition) 124 | (vec) 125 | (.assign consumer))) 126 | 127 | 128 | (defn assignment 129 | "Get the set of partitions currently assigned to this consumer." 130 | [^Consumer consumer] 131 | (set (.assignment consumer))) 132 | 133 | 134 | (defn commit-offsets-async! 135 | "Commit offsets returned by the last poll for all subscribed topics and partitions, 136 | or manually specify offsets to commit. 137 | 138 | This is an asynchronous call and will not block. Any errors encountered are either 139 | passed to the callback (if provided) or discarded. 140 | 141 | `offsets` (optional) - commit the specified offsets for the specified list of topics 142 | and partitions to Kafka. A seq of offset maps, as below: 143 | 144 | e.g. {:topic \"foo\" 145 | :partition 1 146 | :offset 42} 147 | 148 | optionally provide metadata: 149 | 150 | e.g. {:topic \"bar\" 151 | :partition 0 152 | :offset 17 153 | :metadata \"that's so meta.\"} 154 | 155 | The committed offset should be the next message your application will consume, 156 | i.e. `lastProcessedMessageOffset` + 1. 157 | " 158 | ([^Consumer consumer] 159 | (.commitAsync consumer)) 160 | ([^Consumer consumer callback] 161 | (.commitAsync consumer (reify-occ callback))) 162 | ([^Consumer consumer offsets callback] 163 | (let [m (into {} (for [{:keys [topic partition offset metadata]} offsets] 164 | [(topic-partition topic partition) 165 | (if metadata 166 | (offset-and-metadata offset metadata) 167 | (offset-and-metadata offset))]))] 168 | (.commitAsync consumer m (reify-occ callback))))) 169 | 170 | 171 | (defn commit-offsets! 172 | "Commit offsets returned by the last poll for all subscribed topics and partitions, or 173 | manually specify offsets to commit. 174 | 175 | `offsets` (optional) - commit the specified offsets for the specified list of topics 176 | and partitions to Kafka. A seq of offset maps, as below: 177 | 178 | e.g. {:topic \"foo\" 179 | :partition 1 180 | :offset 42} 181 | 182 | optionally provide metadata: 183 | 184 | e.g. {:topic \"bar\" 185 | :partition 0 186 | :offset 17 187 | :metadata \"that's so meta.\"}" 188 | ([^Consumer consumer] 189 | (.commitSync consumer)) 190 | ([^Consumer consumer offsets] 191 | (let [m (into {} (for [{:keys [topic partition offset metadata]} offsets] 192 | [(topic-partition topic partition) 193 | (if metadata 194 | (offset-and-metadata offset metadata) 195 | (offset-and-metadata offset))]))] 196 | (.commitSync consumer m)))) 197 | 198 | 199 | (defn committed 200 | "Return `OffsetAndMetadata` of the last committed offset for the given partition. This 201 | offset will be used as the position for the consumer in the event of a failure. If no 202 | offsets have been committed, return `nil`." 203 | [^Consumer consumer ^String topic ^Integer partition] 204 | (when-let [offset (.committed consumer (topic-partition topic partition))] 205 | (let [m {:offset (.offset offset) 206 | :metadata (.metadata offset)}] 207 | (if (clojure.string/blank? (:metadata m)) 208 | (assoc m :metadata nil) 209 | m)))) 210 | 211 | 212 | (defn pause 213 | "Suspend fetching for a seq of topic name, partition number pairs." 214 | [^Consumer consumer topic partition & tps] 215 | (->> tps 216 | (->tps "pause" topic partition) 217 | (.pause consumer))) 218 | 219 | 220 | (defn poll 221 | "Return a seq of consumer records currently available to the consumer (via a single poll). 222 | Fetches sequetially from the last consumed offset. 223 | 224 | A consumer record is represented as a clojure map with corresponding keys: 225 | `:value`, `:key`, `:partition`, `:topic`, `:offset` 226 | 227 | `timeout` - the time, in milliseconds, spent waiting in poll if data is not 228 | available. If 0, returns immediately with any records that are available now. 229 | Must not be negative." 230 | ([consumer] (poll consumer 100)) 231 | ([consumer timeout] 232 | (->> (.poll consumer timeout) 233 | (map consumer-record->map) 234 | (seq)))) 235 | 236 | 237 | (defn position 238 | "Return the offset of the next record that will be fetched if a record with that offset exists." 239 | ^Long 240 | [^Consumer consumer topic partition] 241 | (.position consumer (topic-partition topic partition))) 242 | 243 | 244 | (defn records 245 | "Return a lazy sequence of sequences of consumer-records by polling the consumer. 246 | 247 | Each element in the returned sequence is the seq of consumer records returned from a 248 | poll by the consumer. The consumer fetches sequetially from the last consumed offset. 249 | 250 | A consumer record is represented as a clojure map with corresponding keys: 251 | `:value`, `:key`, `:partition`, `:topic`, `:offset` 252 | 253 | `timeout` - the time, in milliseconds, spent waiting in poll if data is not 254 | available. If 0, returns immediately with any records that are available now. 255 | Must not be negative." 256 | ([^Consumer consumer] (records consumer 100)) 257 | ([^Consumer consumer timeout] (repeatedly #(poll consumer timeout)))) 258 | 259 | 260 | (defn resume 261 | "Resume specified partitions which have been paused." 262 | [^Consumer consumer topic partition & tps] 263 | (->> tps 264 | (->tps "resume" topic partition) 265 | (.resume consumer))) 266 | 267 | 268 | (defn seek! 269 | "Overrides the fetch offsets that the consumer will use on the next poll." 270 | [^Consumer consumer topic partition offset] 271 | (.seek consumer (topic-partition topic partition) offset)) 272 | 273 | 274 | (defn seek-to! 275 | "Seek to the `:beginning` or `:end` offset for each of the given partitions." 276 | [consumer offset topic partition & tps] 277 | (assert (contains? #{:beginning :end} offset) "offset must be :beginning or :end") 278 | (let [tps (->tps "seek-to!" topic partition tps)] 279 | (case offset 280 | :beginning (.seekToBeginning consumer tps) 281 | :end (.seekToEnd consumer tps)))) 282 | 283 | 284 | (defn subscribe 285 | "Subscribe to the given list of topics to get dynamically assigned partitions. Topic 286 | subscriptions are not incremental. This list will replace the current assignment (if 287 | there is one). It is not possible to combine topic subscription with group management 288 | with manual partition assignment through assign(List). If the given list of topics is 289 | empty, it is treated the same as unsubscribe. 290 | 291 | `topics-or-regex` can be a list of topic names or a `java.util.regex.Pattern` object to 292 | subscribe to all topics matching a specified pattern. 293 | 294 | the optional functions are a callback interface to trigger custom actions when the set 295 | of partitions assigned to the consumer changes." 296 | [^Consumer consumer topics-or-regex & [partitions-assigned-fn partitions-revoked-fn]] 297 | (.subscribe consumer topics-or-regex 298 | (reify-crl partitions-assigned-fn partitions-revoked-fn))) 299 | 300 | 301 | (defn subscription 302 | "Get the current subscription for this consumer." 303 | [^Consumer consumer] 304 | (set (.subscription consumer))) 305 | 306 | 307 | (defn unsubscribe 308 | "Unsubscribe from topics currently subscribed with subscribe. This also clears any 309 | partitions directly assigned through assign." 310 | [^Consumer consumer] 311 | (.unsubscribe consumer)) 312 | 313 | 314 | (defn wakeup 315 | "Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long 316 | poll. The thread which is blocking in an operation will throw `WakeupException`." 317 | [^Consumer consumer] 318 | (.wakeup consumer)) 319 | 320 | 321 | (defn consumer 322 | "Return a `KafkaConsumer`. 323 | 324 | Args: 325 | - `servers`: comma-separated host:port strs or list of strs as bootstrap servers. 326 | - `group-id`: str that identifies the consumer group this consumer belongs to. 327 | - `topics`: (optional) list of topics to which the consumer will be dynamically subscribed. 328 | - `config`: (optional) map of str to str containing additional consumer configuration. 329 | More info on optional config is available here: 330 | http://kafka.apache.org/documentation.html#newconsumerconfigs 331 | 332 | The `StringDeserializer` class is the default for both `key.deserializer` and 333 | `value.deserializer`. 334 | " 335 | ^KafkaConsumer 336 | ([servers group-id] (consumer servers group-id [] {})) 337 | ([servers group-id topics] (consumer servers group-id topics {})) 338 | ([servers group-id topics config] 339 | (let [servers (if (sequential? servers) (str/join "," servers) servers) 340 | c (-> {"bootstrap.servers" servers 341 | "group.id" group-id 342 | "key.deserializer" str-deserializer 343 | "value.deserializer" str-deserializer} 344 | (merge config) 345 | (as-properties) 346 | (KafkaConsumer.))] 347 | (when (not-empty topics) 348 | (subscribe c topics)) 349 | c))) 350 | 351 | ;;;;;;;;;;;;;;;;;;;; 352 | ;; Kafka Producer ;; 353 | ;;;;;;;;;;;;;;;;;;;; 354 | 355 | 356 | (defn flush 357 | "Invoking this method makes all buffered records immediately available to send (even if 358 | `linger.ms` is greater than 0) and blocks on the completion of the requests associated 359 | with these records." 360 | [^Producer producer] 361 | (.flush producer)) 362 | 363 | 364 | (defn ->producer-record 365 | ^ProducerRecord 366 | ([^String topic value] 367 | (ProducerRecord. topic value)) 368 | ([^String topic key value] 369 | (ProducerRecord. topic key value)) 370 | ([^String topic ^Integer partition key value] 371 | (ProducerRecord. topic (int partition) key value)) 372 | ([^String topic ^Integer partition ^Long timestamp key value] 373 | (ProducerRecord. topic (int partition) (long timestamp) key value))) 374 | 375 | 376 | (defn- send-record 377 | [^Producer producer ^ProducerRecord record & [callback]] 378 | (if callback 379 | (.send producer record 380 | (reify Callback 381 | (onCompletion [this metadata ex] 382 | (try 383 | (callback (when metadata 384 | {:offset (.offset metadata) 385 | :partition (.partition metadata) 386 | :topic (.topic metadata)}) 387 | ex) 388 | (catch Exception _ nil))))) 389 | (.send producer record))) 390 | 391 | 392 | (defn send 393 | "Asynchronously send a record to a topic, providing at least a topic and value." 394 | ^java.util.concurrent.Future 395 | ([^Producer producer ^String topic value] 396 | (send-record producer (->producer-record topic value))) 397 | ([^Producer producer ^String topic key value] 398 | (send-record producer (->producer-record topic key value))) 399 | ([^Producer producer ^String topic ^Integer partition key value] 400 | (send-record producer (->producer-record topic partition key value))) 401 | ([^Producer producer ^String topic ^Integer partition ^Long timestamp key value] 402 | (send-record producer (->producer-record topic partition timestamp key value)))) 403 | 404 | 405 | (defn send-then 406 | "Asynchronously send a record to a topic, providing at least a topic and value, and invoke the 407 | provided callback when the send has been acknowledged. 408 | 409 | The callback function should take 2 args: 410 | - a metadata map: the metadata for the record that was sent. 411 | Keys are `:topic`, `:partition`, `:offset`. 412 | - a `java.lang.Exception` object: the exception thrown during processing of this record." 413 | ^java.util.concurrent.Future 414 | ([^Producer producer ^String topic value callback] 415 | (send-record producer (->producer-record topic value) callback)) 416 | ([^Producer producer ^String topic key value callback] 417 | (send-record producer (->producer-record topic key value) callback)) 418 | ([^Producer producer ^String topic ^Integer partition key value callback] 419 | (send-record producer (->producer-record topic (int partition) key value) callback)) 420 | ([^Producer producer ^String topic ^Integer partition ^Long timestamp key value callback] 421 | (send-record producer (->producer-record topic (int partition) (long timestamp) key value) callback))) 422 | 423 | 424 | (defn producer 425 | "Return a `KafkaProducer`. 426 | 427 | The producer is thread safe and sharing a single producer instance across 428 | threads will generally be faster than having multiple instances. 429 | 430 | Args: 431 | - `servers`: comma-separated host:port strs or list of strs as bootstrap servers 432 | - `config`: (optional) a map of str to str containing additional producer configuration. 433 | More info on optional config is available here: 434 | http://kafka.apache.org/documentation.html#producerconfigs 435 | 436 | The `StringSerializer` class is the default for both `key.serializer` and `value.serializer`" 437 | ^KafkaProducer 438 | ([servers] (producer servers {})) 439 | ([servers config] 440 | (-> {"bootstrap.servers" servers 441 | "key.serializer" str-serializer 442 | "value.serializer" str-serializer} 443 | (merge config) 444 | (as-properties) 445 | (KafkaProducer.)))) 446 | 447 | ;;;;;;;;;;;;;;;;;;;;;; 448 | ;; Topic Management ;; 449 | ;;;;;;;;;;;;;;;;;;;;;; 450 | 451 | 452 | (defn- validate-zookeeper-config 453 | "A helper for validating a Zookeeper configuration map and applying default values. Any 454 | invalid item in the provided config will result in an assertion failure. 455 | 456 | Args: 457 | - `config-map`: a map with Zookeeper connection details. Valid keys are as follows: 458 | 459 | `:connection-string` a valid connection string for the Zookeeper instance to connect to. 460 | `:session-timeout` (optional) the session timeout in millis. 461 | `:connect-timeout` (optional) the connect timeout in millis." 462 | [config-map] 463 | (let [defaults {:connection-string nil 464 | :session-timeout 10000 465 | :connect-timeout 10000} 466 | merged (merge defaults config-map) 467 | valid-keys (set (keys defaults)) 468 | merged-keys (set (keys merged))] 469 | (assert (set/subset? merged-keys valid-keys) 470 | (format "Invalid Zookeeper config: %s" 471 | (str/join ", " (set/difference merged-keys valid-keys)))) 472 | (assert (string? (:connection-string merged)) 473 | "Zookeeper config must contain a valid :connection-string") 474 | (assert (integer? (:session-timeout merged)) 475 | "Zookeeper :session-timeout must be an integer") 476 | (assert (integer? (:connect-timeout merged)) 477 | "Zookeeper :connect-timeout must be an integer") 478 | merged)) 479 | 480 | 481 | (defmacro with-zookeeper 482 | "A utility macro for interacting with Zookeeper. 483 | 484 | Args: 485 | - `zk-config`: a map with Zookeeper connection details. This will be validated using 486 | `validate-zookeeper-config` before use. 487 | - `zookeeper`: this will be bound to an instance of `ZkUtils` while the body is executed. 488 | The connection to Zookeeper will be cleaned up when the body exits." 489 | [zk-config zookeeper & body] 490 | `(let [zk-config# (validate-zookeeper-config ~zk-config) 491 | client-and-conn# (ZkUtils/createZkClientAndConnection 492 | (:connection-string zk-config#) 493 | (:session-timeout zk-config#) 494 | (:connect-timeout zk-config#))] 495 | (with-open [client# (._1 client-and-conn#) 496 | connection# (._2 client-and-conn#)] 497 | (let [~zookeeper (ZkUtils. client# connection# false)] 498 | ~@body)))) 499 | 500 | 501 | (def rack-aware-modes 502 | {:disabled (kafka.admin.RackAwareMode$Disabled$.) 503 | :enforced (kafka.admin.RackAwareMode$Enforced$.) 504 | :safe (kafka.admin.RackAwareMode$Safe$.)}) 505 | 506 | 507 | (defn- rack-aware-mode-constant 508 | "Convert a keyword name for a `RackAwareMode` into the appropriate constant from the 509 | underlying Kafka library. 510 | 511 | Args: 512 | - `mode`: a keyword of the same name as one of the constants in `kafka.admin.RackAwareMode`." 513 | [mode] 514 | (when-not (contains? rack-aware-modes mode) 515 | (throw (IllegalArgumentException. (format "Bad RackAwareMode: %s" mode)))) 516 | (get rack-aware-modes mode)) 517 | 518 | 519 | (defn create-topic 520 | "Create a topic. 521 | 522 | Args: 523 | - `zk-config`: a map with Zookeeper connection details as expected by `with-zookeeper`. 524 | - `topic`: the name of the topic to create. 525 | - an unnamed configuration map. Valid keys are as follows: 526 | 527 | `:partitions` (optional) The number of ways to partition the topic. Defaults to 1. 528 | `:replication-factor` (optional) The replication factor for the topic. Defaults to 1. 529 | `:config` (optional) A map of configuration options for the topic. 530 | `:rack-aware-mode` (optional) Control how rack aware replica assignment is done. 531 | Valid values are `:disabled`, `:enforced`, `:safe`. 532 | Default is `:safe`." 533 | [zk-config topic {:keys [partitions replication-factor config rack-aware-mode] 534 | :or {partitions 1 535 | replication-factor 1 536 | config nil 537 | rack-aware-mode :safe}}] 538 | (with-zookeeper zk-config zookeeper 539 | (AdminUtils/createTopic zookeeper 540 | topic 541 | (int partitions) 542 | (int replication-factor) 543 | (as-properties config) 544 | (rack-aware-mode-constant rack-aware-mode)))) 545 | 546 | 547 | (defn topics 548 | "Query existing topics. 549 | 550 | Args: 551 | - `zk-config`: a map with Zookeeper connection details as expected by `with-zookeeper`." 552 | [zk-config] 553 | (with-zookeeper zk-config zookeeper 554 | (-> zookeeper .getAllTopics JavaConversions/seqAsJavaList seq))) 555 | 556 | 557 | (defn topic-exists? 558 | "Query whether or not a topic exists. 559 | 560 | Args: 561 | - `zk-config`: a map with Zookeeper connection details as expected `with-zookeeper`. 562 | - `topic`: The name of the topic to check for." 563 | [zk-config topic] 564 | (with-zookeeper zk-config zookeeper 565 | (AdminUtils/topicExists zookeeper topic))) 566 | 567 | 568 | (defn delete-topic 569 | "Delete a topic. 570 | 571 | Args: 572 | - `zk-config`: A map with Zookeeper connection details as expected by `with-zookeeper`. 573 | - `topic`: The name of the topic to delete." 574 | [zk-config topic] 575 | (with-zookeeper zk-config zookeeper 576 | (AdminUtils/deleteTopic zookeeper topic))) 577 | --------------------------------------------------------------------------------