├── .circleci └── config.yml ├── .gitignore ├── CHANGES.MD ├── LICENSE ├── README.md ├── dev-resources └── benchmarking │ └── results.txt ├── docker-compose.yml ├── project.clj ├── scripts └── release.sh ├── src └── onyx │ ├── kafka │ ├── helpers.clj │ └── information_model.cljc │ ├── plugin │ ├── kafka.clj │ └── partition_assignment.clj │ └── tasks │ └── kafka.clj ├── test-resources └── config.edn └── test └── onyx └── plugin ├── doc_test.clj ├── input_benchmark_test.clj ├── input_broker_reboot_test.clj ├── input_multipartition_scheduling_test.clj ├── input_resume_test.clj ├── input_seek_latest_test.clj ├── input_start_offset_test.clj ├── input_static_partition_test.clj ├── input_test.clj ├── output_bench_test.clj ├── output_test.clj ├── partition_assignment_test.clj ├── target_offsets_test.clj └── test_utils.clj /.circleci/config.yml: -------------------------------------------------------------------------------- 1 | version: 2 2 | executorType: docker 3 | jobs: 4 | test: 5 | docker: 6 | - image: circleci/clojure:lein-2.8.1 7 | - image: zookeeper:3.4.9 8 | environment: 9 | ZOO_MY_ID: 1 10 | ZOO_PORT: 2181 11 | ZOO_SERVERS: server.1=127.0.0.1:2888:3888 12 | - image: confluentinc/cp-kafka:4.0.0 13 | environment: 14 | KAFKA_ADVERTISED_LISTENERS: "PLAINTEXT://127.0.0.1:9092" 15 | KAFKA_ZOOKEEPER_CONNECT: "127.0.0.1:2181" 16 | KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 17 | KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 18 | KAFKA_OFFSET_TOPIC_REPLICATION_FACTOR: 1 19 | KAFKA_BROKER_ID: 1 20 | KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" 21 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 22 | working_directory: ~/repo 23 | steps: 24 | - checkout 25 | - restore_cache: 26 | keys: 27 | - "{{ .Environment.CIRCLE_PROJECT_REPONAME }}-{{ checksum \"project.clj\" }}" 28 | - run: 29 | name: add lein voom 30 | command: mkdir ~/.lein && echo '{:user {:plugins [[lein-voom "0.1.0-20180617_140646-g0ba7ec8"]]}}' > ~/.lein/profiles.clj 31 | - run: 32 | name: lein voom build deps 33 | command: lein voom build-deps 34 | - run: 35 | name: test 36 | command: lein test 37 | - save_cache: 38 | paths: 39 | - ~/.m2 40 | key: "{{ .Environment.CIRCLE_PROJECT_REPONAME }}-{{ checksum \"project.clj\" }}" 41 | release: 42 | docker: 43 | - image: circleci/clojure:lein-2.8.1 44 | working_directory: ~/repo 45 | steps: 46 | - checkout 47 | - restore_cache: 48 | keys: 49 | - "{{ .Environment.CIRCLE_PROJECT_REPONAME }}-{{ checksum \"project.clj\" }}" 50 | - run: 51 | name: add lein voom 52 | command: mkdir ~/.lein && echo '{:user {:plugins [[lein-voom "0.1.0-20180617_140646-g0ba7ec8"]]}}' > ~/.lein/profiles.clj 53 | - run: 54 | name: lein voom build deps 55 | command: lein voom build-deps 56 | - run: 57 | name: lein deploy 58 | command: lein deploy 59 | - save_cache: 60 | paths: 61 | - ~/.m2 62 | key: "{{ .Environment.CIRCLE_PROJECT_REPONAME }}-{{ checksum \"project.clj\" }}" 63 | 64 | workflows: 65 | version: 2 66 | test-build-deploy: 67 | jobs: 68 | - test 69 | - release: 70 | requires: 71 | - test 72 | filters: 73 | branches: 74 | only: master -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /classes 3 | /checkouts 4 | pom.xml 5 | pom.xml.asc 6 | *.jar 7 | *.class 8 | /.lein-* 9 | /.nrepl-port 10 | data/ 11 | onyx.log 12 | hs_err_pid* 13 | scripts/release-scripts 14 | *.iml 15 | /.idea 16 | localrecording.jfr -------------------------------------------------------------------------------- /CHANGES.MD: -------------------------------------------------------------------------------- 1 | #### 0.12.7.0 2 | * Upgrade to kafka 1.0.0. 3 | #### 0.11.0.1 4 | * output plugin: Fix bug where messages would be sent to the wrong partition. 5 | * output plugin: Add support for providing a timestamp in the kafka message. 6 | 7 | #### 0.11.0.0 8 | * Support Onyx level watermarks. 9 | 10 | #### 0.10.0.1 11 | * Switch to plain Kafka producer and consumer. 12 | * Upgrade to Kafka 0.11.0.0. 13 | 14 | #### 0.10.0.0-beta9 15 | * Kafka input plugin now supports reading from multiple partitions with a single peer. 16 | 17 | #### 0.10.0.0-beta2 18 | * Greatly improve kafka output performance. 19 | 20 | #### 0.9.15.1 21 | * Major bug fix: peer recovery may lose messages that were read but were unacked 22 | 23 | #### 0.9.11.1 24 | * Major bug fix: `:kafka/offset-reset` was ignored unless `:kafka/force-reset?` was set. 25 | 26 | #### 0.9.11.0 27 | * **BREAKING CHANGE** `:kafka/offset-reset` `:smallest` was renamed to `:earliest`, `:largest` was renamed to `:latest` 28 | * *WARNING*, task-maps are now schema validated for correctness at run-time. 29 | 30 | #### 0.9.10.1 31 | * **Major bug fix** Resumption of checkpointing was only occurring between jobs using the same `:onyx/tenancy-id`. 32 | * Added ability to pass in arbitrary consumer and producer options via `:kafka/producer-opts` and `:kakfa/consumer-opts` 33 | * Greatly improve performance of kafka write plugin. 34 | * Kill Onyx jobs, rather than rebooting peers, on several unrecoverable plugin errors. 35 | 36 | #### 0.9.10.0 37 | * **BREAKING CHANGE** `:kafka/chan-capacity` has been removed as there is no longer a separate reader thread producing messages 38 | * **BREAKING CHANGE** `:kafka/empty-read-back-off` has been removed as there is no longer a separate reader thread producing messages. Tweak `:onyx/batch-timeout` instead, as this will change the kafka consumer poll time. 39 | * **BREAKING CHANGE** `:kafka/fetch-size` deprecated. Use `:kafka/receive-buffer-bytes`. 40 | * Added `:kafka/start-offsets`, allowing a job to be started at a particular offset on each partition that is being read from. 41 | * Fixed several issues with unclosed ZooKeeper connections 42 | * Large performance improvements were made. 43 | * Moved franzy-admin and franzy-embedded server dependencies to dev dependencies to reduce dependency bloat 44 | 45 | #### 0.9.9.0 46 | * **BREAKING CHANGE** Upgrade to use the new Kafka 0.9 client, deprecating the 0.8 client at [onyx-kafka-0.8](http://www.github.com/onyx-platform/onyx-kafka-0.8). 47 | 48 | #### 0.9.7.0 49 | * Defensively reduce chances of deadlock. 50 | 51 | # 0.9.0.2 52 | * Fixed typo in README 53 | 54 | # 0.8.7 55 | * Add :kafka/wrap-with-metadata? input task-map option. 56 | 57 | # 0.8.6 58 | * Fixed issue where exception was not passed back from the kafka reader thread to read-batch. This meant that peers could become zombies and stop reading from the topic. 59 | 60 | 61 | # 0.7.2.3 62 | Support auto-assigned partitions. 63 | 64 | # 0.7.2.1 65 | * *BREAKING CHANGE* write-messages now takes segments in the form `{:message message-body}` or `{:message message-body :key message-key}`. 66 | * Embedded Kafka server and take-segments test utils function are now included 67 | -------------------------------------------------------------------------------- /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. 215 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## onyx-kafka 2 | 3 | Onyx plugin providing read and write facilities for Kafka. This plugin automatically discovers broker locations from ZooKeeper and updates the consumers when there is a broker failover. 4 | 5 | This plugin version is *only compatible with Kafka 0.10+*. Please use [onyx-kafka-0.8](https://github.com/onyx-platform/onyx-kafka-0.8) with Kafka 0.8. 6 | 7 | #### Installation 8 | 9 | In your project file: 10 | 11 | ```clojure 12 | [org.onyxplatform/onyx-kafka "0.14.5.0"] 13 | ``` 14 | 15 | In your peer boot-up namespace: 16 | 17 | ```clojure 18 | (:require [onyx.plugin.kafka]) 19 | ``` 20 | 21 | #### Functions 22 | 23 | ##### read-messages 24 | 25 | Reads segments from a Kafka topic. Peers will automatically be assigned to each 26 | of the topics partitions, balancing the number of partitions over the number of 27 | peers, unless `:kafka/partition` is supplied in which case only one partition 28 | will be read from. 29 | 30 | Catalog entry: 31 | 32 | ```clojure 33 | {:onyx/name :read-messages 34 | :onyx/plugin :onyx.plugin.kafka/read-messages 35 | :onyx/type :input 36 | :onyx/medium :kafka 37 | :kafka/topic "my topic" 38 | :kafka/receive-buffer-bytes 65536 39 | :kafka/zookeeper "127.0.0.1:2181" 40 | :kafka/offset-reset :earliest 41 | :kafka/deserializer-fn :my.ns/deserializer-fn 42 | :kafka/wrap-with-metadata? false 43 | ;; :kafka/start-offsets {p1 offset1, p2, offset2} 44 | ;; :kafka/target-offsets {p1 offset3, p2, offset4} 45 | :onyx/batch-timeout 50 46 | :onyx/n-peers << NUMBER OF PEERS TO READ PARTITIONS, UP TO N-PARTITION MAX >> 47 | :onyx/batch-size 100 48 | :onyx/doc "Reads messages from a Kafka topic"} 49 | ``` 50 | 51 | Lifecycle entry: 52 | 53 | ```clojure 54 | {:lifecycle/task :read-messages 55 | :lifecycle/calls :onyx.plugin.kafka/read-messages-calls} 56 | ``` 57 | 58 | ###### Attributes 59 | 60 | |key | type | default | description 61 | |-----------------------------|-----------|---------|------------ 62 | |`:kafka/topic` | `string` | | The topic name to connect to 63 | |`:kafka/partition` | `string` | | Optional: partition to read or write to from if auto-assignment is not used 64 | |`:kafka/zookeeper` | `string` | | The ZooKeeper connection string 65 | |`:kafka/offset-reset` | `keyword` | | Offset bound to seek to when not found - `:earliest` or `:latest` 66 | |`:kafka/receive-buffer-bytes`| `integer` |`65536` | The size in the receive buffer in the Kafka consumer. 67 | |`:kafka/key-deserializer-fn` | `keyword` | | A keyword that represents a fully qualified namespaced function to deserialize a record's key. Takes one argument - a byte array. Only used when `:kafka/wrap-with-metadata?` is true. 68 | |`:kafka/deserializer-fn` | `keyword` | | A keyword that represents a fully qualified namespaced function to deserialize a record's value. Takes one argument - a byte array 69 | |`:kafka/wrap-with-metadata?` | `boolean` |`false` | Wraps message into map with keys `:key`, `:serialized-key-size`, `:serialized-value-size`, `:offset`, `:timestamp`, `:partition`, `:topic` and `:message` itself 70 | |`:kafka/start-offsets` | `map` | | Allows a task to be supplied with the starting offsets for all partitions. Maps partition to offset, e.g. `{0 50, 1, 90}` will start at offset 50 for partition 0, and offset 90 for partition 1 71 | |`:kafka/target-offsets` | `map` | | Allows a task to be supplied with target offsets for all partitions. The consumer will read up to and including the target offset for each partition. 72 | |`:kafka/consumer-opts` | `map` | | A map of arbitrary configuration to merge into the underlying Kafka consumer base configuration. Map should contain strings as keys, and the valid values described in the [Kafka Docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). 73 | 74 | ##### write-messages 75 | 76 | Writes segments to a Kafka topic using the Kafka "new" producer. 77 | 78 | Catalog entry: 79 | 80 | ```clojure 81 | {:onyx/name :write-messages 82 | :onyx/plugin :onyx.plugin.kafka/write-messages 83 | :onyx/type :output 84 | :onyx/medium :kafka 85 | :kafka/topic "topic" 86 | :kafka/zookeeper "127.0.0.1:2181" 87 | :kafka/serializer-fn :my.ns/serializer-fn 88 | :kafka/request-size 307200 89 | :onyx/batch-size batch-size 90 | :onyx/doc "Writes messages to a Kafka topic"} 91 | ``` 92 | 93 | Lifecycle entry: 94 | 95 | ```clojure 96 | {:lifecycle/task :write-messages 97 | :lifecycle/calls :onyx.plugin.kafka/write-messages-calls} 98 | ``` 99 | 100 | Segments supplied to a `:onyx.plugin.kafka/write-messages` task should be in in 101 | the following form: `{:message message-body}` with optional partition, topic and 102 | key values. 103 | 104 | ``` clj 105 | {:message message-body 106 | :key optional-key 107 | :partition optional-partition 108 | :topic optional-topic} 109 | ``` 110 | 111 | ###### Attributes 112 | 113 | |key | type | default | description 114 | |----------------------------|-----------|---------|------------ 115 | |`:kafka/topic` | `string` | | The topic name to connect to 116 | |`:kafka/zookeeper` | `string` | | The ZooKeeper connection string 117 | |`:kafka/key-serializer-fn` | `keyword` | | A keyword that represents a fully qualified namespaced function to serialize a record's key. Takes one argument - the segment 118 | |`:kafka/serializer-fn` | `keyword` | | A keyword that represents a fully qualified namespaced function to serialize a record's value. Takes one argument - the segment 119 | |`:kafka/request-size` | `number` |`307200` | The maximum size of request messages. Maps to the `max.request.size` value of the internal kafka producer. 120 | |`:kafka/no-seal?` | `boolean` |`false` | Do not write :done to the topic when task receives the sentinel signal (end of batch job) 121 | |`:kafka/producer-opts` | `map` | | A map of arbitrary configuration to merge into the underlying Kafka producer base configuration. Map should contain strings as keys, and the valid values described in the [Kafka Docs](http://kafka.apache.org/documentation.html#producerconfigs). 122 | 123 | #### Test Utilities 124 | 125 | A take-segments utility function is provided for use when testing the results 126 | of jobs with kafka output tasks. take-segments reads from a topic until a :done 127 | is reached, and then returns the results. Note, if a `:done` is never written to a 128 | topic, this will hang forever as there is no timeout. 129 | 130 | ```clojure 131 | (ns your-ns.a-test 132 | (:require [onyx.kafka.utils :as kpu])) 133 | 134 | ;; insert code to run a job here 135 | 136 | ;; retrieve the segments on the topic 137 | (def results 138 | (kpu/take-segments (:zookeeper/address peer-config) "yourtopic" your-decompress-fn)) 139 | 140 | (last results) 141 | ; :done 142 | 143 | ``` 144 | 145 | #### Development 146 | 147 | To benchmark, start a real ZooKeeper instance (at 127.0.0.1:2181) and Kafka instance, and run the following benchmarks. 148 | 149 | Write perf, single peer writer: 150 | ``` 151 | TIMBRE_LOG_LEVEL=:info lein test onyx.plugin.output-bench-test :benchmark 152 | ``` 153 | 154 | Read perf, single peer reader: 155 | ``` 156 | TIMBRE_LOG_LEVEL=:info lein test onyx.plugin.input-benchmark-test :benchmark 157 | ``` 158 | 159 | Past results are maintained in `dev-resources/benchmarking/results.txt`. 160 | 161 | #### Contributing 162 | 163 | Pull requests into the master branch are welcomed. 164 | 165 | #### License 166 | 167 | Copyright © 2018 Distributed Masonry 168 | 169 | Distributed under the Eclipse Public License, the same as Clojure. 170 | -------------------------------------------------------------------------------- /dev-resources/benchmarking/results.txt: -------------------------------------------------------------------------------- 1 | VERSION, Date, Machine, Type, Results 2 | 0.9.10.1-SNAPSHOT, 20-Sep-2016, 2013 Mac Pro 6 core, output, 79697.15 3 | 0.9.10.1-SNAPSHOT, 20-Sep-2016, 2013 Mac Pro 6 core, input, 89621.8 4 | -------------------------------------------------------------------------------- /docker-compose.yml: -------------------------------------------------------------------------------- 1 | zookeeper: 2 | image: wurstmeister/zookeeper 3 | ports: 4 | - "2181:2181" 5 | kafka: 6 | image: wurstmeister/kafka:0.11.0.0 7 | ports: 8 | - "9092:9092" 9 | links: 10 | - zookeeper:zk 11 | environment: 12 | KAFKA_ADVERTISED_HOST_NAME: 127.0.0.1 13 | KAFKA_ADVERTISED_PORT: 9092 14 | volumes: 15 | - /var/run/docker.sock:/var/run/docker.sock 16 | -------------------------------------------------------------------------------- /project.clj: -------------------------------------------------------------------------------- 1 | (defproject org.onyxplatform/onyx-kafka "0.14.5.0" 2 | :description "Onyx plugin for Kafka" 3 | :url "https://github.com/onyx-platform/onyx-kafka" 4 | :license {:name "Eclipse Public License" 5 | :url "http://www.eclipse.org/legal/epl-v10.html"} 6 | :repositories {"snapshots" {:url "https://clojars.org/repo" 7 | :username :env 8 | :password :env 9 | :sign-releases false} 10 | "releases" {:url "https://clojars.org/repo" 11 | :username :env 12 | :password :env 13 | :sign-releases false}} 14 | :dependencies [[org.clojure/clojure "1.8.0"] 15 | ^{:voom {:repo "git@github.com:onyx-platform/onyx.git" :branch "master"}} 16 | [org.onyxplatform/onyx "0.14.5"] 17 | [org.apache.kafka/kafka_2.11 "1.0.0" :exclusions [org.slf4j/slf4j-log4j12]] 18 | [org.apache.kafka/kafka-clients "1.0.0"] 19 | [com.stuartsierra/component "0.2.3"] 20 | [cheshire "5.7.0"]] 21 | :source-paths ["src/"] 22 | :test-paths ["test/"] 23 | :profiles {:dev {:dependencies [[org.clojure/test.check "0.9.0"] 24 | [com.gfredericks/test.chuck "0.2.7"] 25 | [zookeeper-clj "0.9.3" :exclusions [io.netty/netty org.apache.zookeeper/zookeeper]] 26 | [aero "0.2.0"] 27 | [prismatic/schema "1.0.5"]] 28 | :resource-paths ["test-resources/"] 29 | :plugins [[lein-set-version "0.4.1"] 30 | [lein-update-dependency "0.1.2"] 31 | [lein-pprint "1.1.1"]] 32 | :global-vars {*warn-on-reflection* true 33 | *assert* false 34 | *unchecked-math* :warn-on-boxed} 35 | :java-opts ^:replace ["-server" 36 | "-XX:+UseG1GC" 37 | "-XX:-OmitStackTraceInFastThrow" 38 | "-Xmx2g" 39 | "-Daeron.client.liveness.timeout=50000000000"]}}) 40 | -------------------------------------------------------------------------------- /scripts/release.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -o errexit 4 | set -o pipefail 5 | set -o nounset 6 | set -o xtrace 7 | 8 | # Make sure we're in onyx-kafka's directory first 9 | cd "$(dirname "$0")/.." 10 | 11 | REPO_SRC="https://github.com/onyx-platform/onyx-release-scripts.git" 12 | LOCAL_REPO="scripts/release-scripts" 13 | LOCAL_REPO_VC_DIR=$LOCAL_REPO/.git 14 | 15 | pushd . 16 | 17 | if [ ! -d $LOCAL_REPO_VC_DIR ] 18 | then 19 | git clone $REPO_SRC $LOCAL_REPO 20 | else 21 | cd $LOCAL_REPO 22 | git pull $REPO_SRC 23 | popd 24 | fi 25 | 26 | bash "$LOCAL_REPO/release_plugin.sh" "$@" 27 | -------------------------------------------------------------------------------- /src/onyx/kafka/helpers.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.kafka.helpers 2 | (:require [clojure.string :as string] 3 | [taoensso.timbre :as log]) 4 | (:import [kafka.utils ZkUtils] 5 | [org.apache.kafka.clients.consumer KafkaConsumer ConsumerRecord] 6 | [org.apache.kafka.clients.producer KafkaProducer Callback ProducerRecord] 7 | [org.apache.kafka.common.serialization ByteArrayDeserializer ByteArraySerializer Serializer Deserializer] 8 | [org.apache.kafka.common TopicPartition PartitionInfo] 9 | [kafka.admin AdminUtils] 10 | [kafka.utils ZKStringSerializer$] 11 | [org.I0Itec.zkclient ZkClient ZkConnection IZkConnection] 12 | [org.I0Itec.zkclient.serialize ZkSerializer] 13 | [scala.collection JavaConversions] 14 | [java.util Properties])) 15 | 16 | (def zk-defaults 17 | {:session-timeout 30000 18 | :connection-timeout 30000 19 | :operation-retry-timeout (long -1) 20 | :serializer (ZKStringSerializer$/MODULE$)}) 21 | 22 | (defn munge-servers [servers] 23 | (if (coll? servers) 24 | (string/join "," servers) 25 | servers)) 26 | 27 | (defn as-properties ^Properties [m] 28 | (let [ps (Properties.)] 29 | (doseq [[k v] m] (.setProperty ps k (str v))) 30 | ps)) 31 | 32 | (defn as-java [opts] 33 | (cond-> opts 34 | (get opts "bootstrap.servers") (update "bootstrap.servers" munge-servers))) 35 | 36 | (defn make-zk-connection [zk-config] 37 | (let [{:keys [servers session-timeout]} zk-config 38 | server-seq (if (coll? servers) (string/join "," servers) servers)] 39 | (ZkConnection. server-seq session-timeout))) 40 | 41 | (defn make-zk-client [zk-connection zk-config] 42 | (let [{:keys [connection-timeout operation-retry-timeout serializer]} zk-config] 43 | (ZkClient. ^IZkConnection zk-connection 44 | (int connection-timeout) 45 | ^ZkSerializer serializer 46 | (long operation-retry-timeout)))) 47 | 48 | (defn make-zk-utils ^ZkUtils [zk-config secure?] 49 | (let [merged-config (merge zk-defaults zk-config) 50 | session-timeout (:session-timeout merged-config) 51 | connection-timeout (:connection-timeout merged-config) 52 | zk-connection (make-zk-connection merged-config) 53 | zk-client (make-zk-client zk-connection merged-config)] 54 | (ZkUtils. zk-client zk-connection secure?))) 55 | 56 | (defn all-brokers 57 | [^ZkUtils zk-utils] 58 | (->> (.getAllBrokersInCluster zk-utils) 59 | (JavaConversions/bufferAsJavaList) 60 | (map 61 | (fn [broker] 62 | {:id (.id broker) 63 | :endpoints 64 | (map 65 | (fn [endpoint] 66 | {:host (.host endpoint) 67 | :port (.port endpoint) 68 | :protocol-type (.name (.securityProtocol endpoint))}) 69 | (JavaConversions/seqAsJavaList (.endPoints broker)))})))) 70 | 71 | (defn id->broker [zk-addr] 72 | (with-open [zk-utils (make-zk-utils {:servers zk-addr} false)] 73 | (reduce 74 | (fn [result {:keys [id endpoints]}] 75 | (assoc 76 | result 77 | id 78 | (str (:host (first endpoints)) ":" (:port (first endpoints))))) 79 | {} 80 | (all-brokers zk-utils)))) 81 | 82 | (defn byte-array-serializer [] 83 | (ByteArraySerializer.)) 84 | 85 | (defn byte-array-deserializer [] 86 | (ByteArrayDeserializer.)) 87 | 88 | (defn ^KafkaProducer build-producer [producer-opts key-serializer value-serializer] 89 | (KafkaProducer. ^Properties (as-properties (as-java producer-opts)) 90 | ^Serializer key-serializer 91 | ^Serializer value-serializer)) 92 | 93 | (defn ^KafkaConsumer build-consumer [consumer-opts key-deserializer value-deserializer] 94 | (KafkaConsumer. ^Properties (as-properties (as-java consumer-opts)) 95 | ^Deserializer key-deserializer 96 | ^Deserializer value-deserializer)) 97 | 98 | (defn partitions-for-topic [consumer topic] 99 | (let [parts (.partitionsFor ^KafkaConsumer consumer topic)] 100 | (map 101 | (fn [part] 102 | {:topic (.topic part) 103 | :partition (.partition part)}) 104 | parts))) 105 | 106 | (defn to-topic-partition [tp] 107 | (TopicPartition. (:topic tp) (:partition tp))) 108 | 109 | (defn assign-partitions! [consumer topic-partitions] 110 | (->> topic-partitions 111 | (mapv to-topic-partition) 112 | (.assign ^KafkaConsumer consumer))) 113 | 114 | (defn seek-to-offset! [consumer topic-partition offset] 115 | (let [encoded (to-topic-partition topic-partition)] 116 | (.seek ^KafkaConsumer consumer encoded offset))) 117 | 118 | (defn seek-to-beginning! [consumer topic-partitions] 119 | (let [encoded (map to-topic-partition topic-partitions)] 120 | (.seekToBeginning ^KafkaConsumer consumer encoded))) 121 | 122 | (defn seek-to-end! [consumer topic-partitions] 123 | (let [encoded (map to-topic-partition topic-partitions)] 124 | (.seekToEnd ^KafkaConsumer consumer encoded))) 125 | 126 | (defn consumer-record->message 127 | [decompress-fn ^ConsumerRecord m] 128 | {:key (some-> m (.key) decompress-fn) 129 | :partition (.partition m) 130 | :topic (.topic m) 131 | :value (-> m (.value) decompress-fn) 132 | :timestamp (.timestamp m)}) 133 | 134 | (defn poll! [consumer timeout] 135 | (.poll ^KafkaConsumer consumer timeout)) 136 | 137 | (defn take-now 138 | "Reads whatever it can from a topic on the assumption that we've distributed 139 | work across multiple topics and another topic contained :done." 140 | ([bootstrap-servers topic decompress-fn] 141 | (take-now bootstrap-servers topic decompress-fn 5000)) 142 | ([bootstrap-servers topic decompress-fn timeout] 143 | (log/info {:msg "Taking now..." :topic topic}) 144 | (let [c (build-consumer {"bootstrap.servers" bootstrap-servers} (byte-array-deserializer) (byte-array-deserializer)) 145 | topic-partitions [{:topic topic :partition 0}]] 146 | (assign-partitions! c topic-partitions) 147 | (seek-to-beginning! c topic-partitions) 148 | (mapv #(consumer-record->message decompress-fn %) (poll! c timeout))))) 149 | 150 | (defn create-topic! [zk-addr topic-name num-partitions replication-factor] 151 | (with-open [zk-utils (make-zk-utils {:servers zk-addr} false)] 152 | (AdminUtils/createTopic 153 | zk-utils topic-name num-partitions replication-factor (as-properties {}) 154 | (kafka.admin.RackAwareMode$Safe$.)))) 155 | 156 | (deftype ProducerCallback [p] 157 | Callback 158 | (onCompletion [_ v exception] 159 | (deliver p true))) 160 | 161 | (defn send-sync! [producer topic part k v] 162 | (let [p (promise) 163 | record (ProducerRecord. topic part k v)] 164 | (.send producer record (->ProducerCallback p)) 165 | @p)) 166 | 167 | (defn partition-info->topic-partition [topic ^PartitionInfo part-info] 168 | (TopicPartition. topic (.partition part-info))) 169 | 170 | (defn end-offsets [bootstrap-servers topic] 171 | (let [opts {"bootstrap.servers" bootstrap-servers} 172 | k-deser (ByteArrayDeserializer.) 173 | v-deser (ByteArrayDeserializer.)] 174 | (with-open [consumer (build-consumer opts k-deser v-deser)] 175 | (let [parts (.partitionsFor consumer topic) 176 | tps (map (partial partition-info->topic-partition topic) parts)] 177 | (.endOffsets consumer tps))))) 178 | 179 | (defn offsets->clj [end-offsets] 180 | (reduce-kv 181 | (fn [all ^TopicPartition k v] 182 | (assoc all (.partition k) v)) 183 | {} 184 | (into {} end-offsets))) 185 | 186 | (defn beginning-end-offsets-clj [bootstrap-servers topic] 187 | (let [opts {"bootstrap.servers" bootstrap-servers} 188 | k-deser (ByteArrayDeserializer.) 189 | v-deser (ByteArrayDeserializer.)] 190 | (with-open [consumer (build-consumer opts k-deser v-deser)] 191 | (let [parts (.partitionsFor consumer topic) 192 | tps (map (partial partition-info->topic-partition topic) parts)] 193 | {:beginning-offsets (offsets->clj (.beginningOffsets consumer tps)) 194 | :end-offsets (offsets->clj (.endOffsets consumer tps))})))) 195 | -------------------------------------------------------------------------------- /src/onyx/kafka/information_model.cljc: -------------------------------------------------------------------------------- 1 | (ns onyx.kafka.information-model) 2 | 3 | (def model 4 | {:catalog-entry 5 | {:onyx.plugin.kafka/read-messages 6 | {:summary "An input task to read messages from a Kafka topic." 7 | :model {:kafka/topic 8 | {:doc "The topic name to read from." 9 | :type :string} 10 | 11 | :kafka/group-id 12 | {:doc "The consumer identity to store in ZooKeeper." 13 | :type :string} 14 | 15 | :kafka/partition 16 | {:doc "Partition to read from if auto-assignment is not used." 17 | :type :string 18 | :optional? true} 19 | 20 | :kafka/bootstrap-servers 21 | {:doc "The list of Kafka brokers to use for connection bootstrap." 22 | :type [:string] 23 | :optional? true} 24 | 25 | :kafka/zookeeper 26 | {:doc "The ZooKeeper connection string." 27 | :type :string 28 | :optional? true} 29 | 30 | :kafka/offset-reset 31 | {:doc "Offset bound to seek to when not found - `:earliest` or `:latest`." 32 | :choices [:earliest :latest] 33 | :type :keyword} 34 | 35 | :kafka/force-reset? 36 | {:doc "Force to read from the beginning or end of the log, as specified by `:kafka/offset-reset`. If false, reads from the last acknowledged messsage if it exists." 37 | :deprecation-version "0.9.10.0" 38 | :deprecation-doc ":kafka/force-reset? deprecated as this functionality has been subsumed by onyx resume-point." 39 | :type :boolean} 40 | 41 | :kafka/chan-capacity 42 | {:doc "The buffer size of the Kafka reading channel." 43 | :type :long 44 | :default 1000 45 | :deprecation-version "0.9.10.0" 46 | :deprecation-doc ":kafka/chan-capacity deprecated as onyx-kafka no longer uses a separate producer thread." 47 | :optional? true} 48 | 49 | :kafka/receive-buffer-bytes 50 | {:doc "The size in the receive buffer in the Kafka consumer." 51 | :type :long 52 | :default 65536 53 | :optional? true} 54 | 55 | :kafka/consumer-opts 56 | {:doc "A map of arbitrary configuration to merge into the underlying Kafka consumer base configuration. Map should contain keywords as keys, and the valid values described in the [Kafka Docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). Please note that key values such as `fetch.min.bytes` must be in keyword form, i.e. `:fetch.min.bytes`." 57 | :type :map 58 | :optional? true} 59 | 60 | :kafka/fetch-size 61 | {:doc "The size in bytes to request from ZooKeeper per fetch request." 62 | :type :long 63 | :default 307200 64 | :deprecation-version "0.9.10.0" 65 | :deprecation-doc ":kafka/fetch-size deprecated. Use :kafka/receive-buffer-bytes instead." 66 | :optional? true} 67 | 68 | :kafka/empty-read-back-off 69 | {:doc "The amount of time to back off between reads when nothing was fetched from a consumer." 70 | :type :long 71 | :default 500 72 | :deprecation-version "0.9.10.0" 73 | :deprecation-doc ":kafka/empty-read-back-off deprecated in lieu of better use of :onyx/batch-timeout" 74 | :optional? true} 75 | 76 | :kafka/commit-interval 77 | {:doc "The interval in milliseconds to commit the latest acknowledged offset to ZooKeeper." 78 | :type :long 79 | :default 2000 80 | :optional? true} 81 | 82 | :kafka/deserializer-fn 83 | {: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." 84 | :type :keyword} 85 | 86 | :kafka/key-deserializer-fn 87 | {: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." 88 | :type :keyword 89 | :optional? true} 90 | 91 | :kafka/wrap-with-metadata? 92 | {:doc "Wraps message into map with keys `:offset`, `:partitions`, `:topic` and `:message` itself." 93 | :type :boolean 94 | :default false 95 | :optional? true} 96 | 97 | :kafka/start-offsets 98 | {:doc "Allows a task to be supplied with the starting offsets for all partitions. Maps partition to offset, e.g. `{0 50, 1, 90}` will start at offset 50 for partition 0, and offset 90 for partition 1." 99 | :type :map 100 | :optional? true} 101 | 102 | :kafka/target-offsets 103 | {:doc "Allows a task to be supplied with target offsets for all partitions. The consumer will read up to and including the target offset for each partition." 104 | :type :map 105 | :optional? true}}} 106 | 107 | :onyx.plugin.kafka/write-messages 108 | {:summary "Write messages to kafka." 109 | :model {:kafka/topic 110 | {:doc "The topic name to write to. Must either be supplied or otherwise all messages must contain a `:topic` key" 111 | :optional? true 112 | :type :string} 113 | 114 | :kafka/partition 115 | {:doc "Partition to write to, if you do not wish messages to be auto allocated to partitions. Must either be supplied in the task map, or all messages should contain a `:partition` key." 116 | :type :string 117 | :optional? true} 118 | 119 | :kafka/bootstrap-servers 120 | {:doc "The list of Kafka brokers to use for connection bootstrap." 121 | :type [:string] 122 | :optional? true} 123 | 124 | :kafka/zookeeper 125 | {:doc "The ZooKeeper connection string." 126 | :type :string 127 | :optional? true} 128 | 129 | :kafka/request-size 130 | {:doc "The maximum size of request messages. Maps to the `max.request.size` value of the internal kafka producer." 131 | :type :long 132 | :optional? true} 133 | 134 | :kafka/serializer-fn 135 | {:doc "A keyword that represents a fully qualified namespaced function to serialize a record's value. Takes one argument - the segment." 136 | :type :keyword} 137 | 138 | :kafka/key-serializer-fn 139 | {:doc "A keyword that represents a fully qualified namespaced function to serialize a record's key. Takes one argument - the segment." 140 | :type :keyword 141 | :optional? true} 142 | 143 | :kafka/producer-opts 144 | {:doc "A map of arbitrary configuration to merge into the underlying Kafka producer base configuration. Map should contain keywords as keys, and the valid values described in the [Kafka Docs](http://kafka.apache.org/documentation.html#producerconfigs). Please note that key values such as `buffer.memory` must be in keyword form, i.e. `:buffer.memory`." 145 | :type :map 146 | :optional? true} 147 | 148 | :kafka/no-seal? 149 | {:doc "Do not write :done to the topic when task receives the sentinel signal (end of batch job)." 150 | :type :boolean 151 | :default false 152 | :optional? true}}}} 153 | 154 | :lifecycle-entry 155 | {:onyx.plugin.kafka/read-messages 156 | {:model 157 | [{:task.lifecycle/name :read-messages 158 | :lifecycle/calls :onyx.plugin.kafka/read-messages-calls}]} 159 | 160 | :onyx.plugin.kafka/write-messages 161 | {:model 162 | [{:task.lifecycle/name :write-messages 163 | :lifecycle/calls :onyx.plugin.kafka/write-messages-calls}]}} 164 | 165 | :display-order 166 | {:onyx.plugin.kafka/read-messages 167 | [:kafka/topic 168 | :kafka/partition 169 | :kafka/group-id 170 | :kafka/zookeeper 171 | :kafka/bootstrap-servers 172 | :kafka/offset-reset 173 | :kafka/force-reset? 174 | :kafka/deserializer-fn 175 | :kafka/key-deserializer-fn 176 | :kafka/receive-buffer-bytes 177 | :kafka/commit-interval 178 | :kafka/wrap-with-metadata? 179 | :kafka/start-offsets 180 | :kafka/target-offsets 181 | :kafka/consumer-opts 182 | :kafka/empty-read-back-off 183 | :kafka/fetch-size 184 | :kafka/chan-capacity] 185 | 186 | :onyx.plugin.kafka/write-messages 187 | [:kafka/topic 188 | :kafka/zookeeper 189 | :kafka/bootstrap-servers 190 | :kafka/partition 191 | :kafka/serializer-fn 192 | :kafka/key-serializer-fn 193 | :kafka/request-size 194 | :kafka/no-seal? 195 | :kafka/producer-opts]}}) 196 | -------------------------------------------------------------------------------- /src/onyx/plugin/kafka.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.kafka 2 | (:require [onyx.plugin.partition-assignment :refer [partitions-for-slot]] 3 | [onyx.kafka.helpers :as h] 4 | [taoensso.timbre :as log :refer [fatal info]] 5 | [onyx.static.default-vals :refer [arg-or-default]] 6 | [onyx.plugin.protocols :as p] 7 | [onyx.static.util :refer [kw->fn]] 8 | [onyx.tasks.kafka] 9 | [schema.core :as s]) 10 | (:import [java.util.concurrent.atomic AtomicLong] 11 | [org.apache.kafka.clients.consumer ConsumerRecords ConsumerRecord] 12 | [org.apache.kafka.clients.consumer KafkaConsumer ConsumerRebalanceListener Consumer] 13 | [org.apache.kafka.common.record TimestampType] 14 | [org.apache.kafka.common TopicPartition] 15 | [org.apache.kafka.common.metrics Metrics] 16 | [org.apache.kafka.clients.producer Callback KafkaProducer ProducerRecord])) 17 | 18 | (def defaults 19 | {:kafka/receive-buffer-bytes 65536 20 | :kafka/wrap-with-metadata? false 21 | :kafka/unable-to-find-broker-backoff-ms 8000}) 22 | 23 | (defn seek-offset! [log-prefix consumer kpartitions task-map topic checkpoint] 24 | (let [policy (:kafka/offset-reset task-map) 25 | start-offsets (:kafka/start-offsets task-map)] 26 | (doseq [kpartition kpartitions] 27 | (cond (get checkpoint kpartition) 28 | (let [offset (get checkpoint kpartition)] 29 | (info log-prefix "Seeking to checkpointed offset at:" (inc offset)) 30 | (h/seek-to-offset! consumer {:topic topic :partition kpartition} (inc offset))) 31 | 32 | start-offsets 33 | (let [offset (get start-offsets kpartition)] 34 | (when-not offset 35 | (throw (ex-info "Offset missing for existing partition when using :kafka/start-offsets" 36 | {:missing-partition kpartition 37 | :kafka/start-offsets start-offsets}))) 38 | (h/seek-to-offset! consumer {:topic topic :partition kpartition} offset)) 39 | 40 | (= policy :earliest) 41 | (do 42 | (info log-prefix "Seeking to earliest offset on topic" {:topic topic :partition kpartition}) 43 | (h/seek-to-beginning! consumer [{:topic topic :partition kpartition}])) 44 | 45 | (= policy :latest) 46 | (do 47 | (info log-prefix "Seeking to latest offset on topic" {:topic topic :partition kpartition}) 48 | (h/seek-to-end! consumer [{:topic topic :partition kpartition}])) 49 | 50 | :else 51 | (throw (ex-info "Tried to seek to unknown policy" {:recoverable? false 52 | :policy policy})))))) 53 | 54 | (defn find-brokers [task-map] 55 | (let [zk-addr (:kafka/zookeeper task-map) 56 | _ (when-not zk-addr 57 | (throw (ex-info "Either :kafka/bootstrap-servers or :kafka/zookeeper must be defined." 58 | {:task-map task-map}))) 59 | results (vals (h/id->broker zk-addr))] 60 | (if (seq results) 61 | results 62 | (do 63 | (info "Could not locate any Kafka brokers to connect to. Backing off.") 64 | (Thread/sleep (or (:kafka/unable-to-find-broker-backoff-ms task-map) 65 | (:kafka/unable-to-find-broker-backoff-ms defaults))) 66 | (throw (ex-info "Could not locate any Kafka brokers to connect to." 67 | {:recoverable? true 68 | :zk-addr zk-addr})))))) 69 | 70 | (defprotocol PluginMeta 71 | (metadata [this])) 72 | 73 | (defn start-kafka-consumer 74 | [event lifecycle] 75 | {}) 76 | 77 | (defn check-num-peers-equals-partitions 78 | [{:keys [onyx/min-peers onyx/max-peers onyx/n-peers kafka/partition] :as task-map} n-partitions] 79 | (let [fixed-partition? (and partition (or (= 1 n-peers) 80 | (= 1 max-peers))) 81 | fixed-npeers? (or (and min-peers (= min-peers max-peers)) 82 | (= 1 max-peers) 83 | (and n-peers (and (not min-peers) (not max-peers)))) 84 | n-peers (or max-peers n-peers) 85 | n-peers-less-eq-n-partitions (and n-peers (<= n-peers n-partitions))] 86 | (when-not (or fixed-partition? fixed-npeers? n-peers-less-eq-n-partitions) 87 | (let [e (ex-info ":onyx/min-peers must equal :onyx/max-peers, or :onyx/n-peers must be set, and :onyx/min-peers and :onyx/max-peers must not be set. Number of peers should also be less than or equal to the number of partitions." 88 | {:n-partitions n-partitions 89 | :n-peers n-peers 90 | :min-peers min-peers 91 | :max-peers max-peers 92 | :recoverable? false 93 | :task-map task-map})] 94 | (log/error e) 95 | (throw e))))) 96 | 97 | (defn assign-partitions-to-slot! [consumer* task-map topic n-partitions slot] 98 | (if-let [part (:partition task-map)] 99 | (let [p (Integer/parseInt part)] 100 | (h/assign-partitions! consumer* [{:topic topic :partition p}]) 101 | [p]) 102 | (let [n-slots (or (:onyx/n-peers task-map) (:onyx/max-peers task-map)) 103 | [lower upper] (partitions-for-slot n-partitions n-slots slot) 104 | parts-range (range lower (inc upper)) 105 | parts (map (fn [p] {:topic topic :partition p}) parts-range)] 106 | (h/assign-partitions! consumer* parts) 107 | parts-range))) 108 | 109 | (defn set-lag! [^AtomicLong lag-gauge ^KafkaConsumer consumer] 110 | (.set lag-gauge 111 | (reduce (fn [lag [tp offset]] 112 | (+ lag (- offset (.position consumer tp)))) 113 | 0 114 | (.endOffsets consumer (.assignment consumer))))) 115 | 116 | (defn all-partitions-paused? 117 | [^KafkaConsumer consumer kpartitions] 118 | (let [paused (into #{} 119 | (map #(.partition ^TopicPartition %)) 120 | (.paused consumer))] 121 | (= paused (set kpartitions)))) 122 | 123 | (defn paused? [^KafkaConsumer consumer part] 124 | (let [paused (map #(.partition ^TopicPartition %) 125 | (.paused consumer))] 126 | (some #{part} paused))) 127 | 128 | (defn current-partition-statuses [checkpoint ^KafkaConsumer consumer target-offsets kpartitions] 129 | (if target-offsets 130 | (let [beginning-offsets (into {} 131 | (map (fn [[^TopicPartition topic-partition offset]] 132 | [(.partition topic-partition) offset]) 133 | (.beginningOffsets consumer (.assignment consumer))))] 134 | (->> kpartitions 135 | (map (fn [p] 136 | (let [current-offset (or (get checkpoint p) (get beginning-offsets p)) 137 | _ (assert current-offset) 138 | target-offset (get target-offsets p) 139 | drained? (and target-offset 140 | (>= current-offset target-offset))] 141 | [p (if drained? :drained :reading)]))) 142 | (into {}))) 143 | (->> kpartitions 144 | (map (fn [p] [p :reading])) 145 | (into {})))) 146 | 147 | (deftype KafkaReadMessages 148 | [log-prefix task-map topic ^:unsynchronized-mutable kpartitions batch-timeout 149 | deserializer-fn segment-fn ^AtomicLong watermark ^AtomicLong lag-gauge ^KafkaConsumer ^:unsynchronized-mutable consumer 150 | ^:unsynchronized-mutable iter ^:unsynchronized-mutable partition->offset drained 151 | target-offsets] 152 | PluginMeta 153 | (metadata [this] 154 | {:consumer consumer 155 | :beginning-offsets (into {} 156 | (map (fn [[^TopicPartition topic-partition offset]] 157 | [{:topic (.topic topic-partition) 158 | :partition (.partition topic-partition)} 159 | offset]) 160 | (.beginningOffsets consumer (.assignment consumer)))) 161 | :end-offsets (into {} 162 | (map (fn [[^TopicPartition topic-partition offset]] 163 | [{:topic (.topic topic-partition) 164 | :partition (.partition topic-partition)} 165 | offset]) 166 | (.endOffsets consumer (.assignment consumer)))) 167 | :partitions kpartitions 168 | :partition->offset partition->offset}) 169 | p/Plugin 170 | (start [this event] 171 | (let [{:keys [kafka/bootstrap-servers kafka/group-id kafka/consumer-opts]} task-map 172 | brokers (or bootstrap-servers (find-brokers task-map)) 173 | _ (s/validate onyx.tasks.kafka/KafkaInputTaskMap task-map) 174 | consumer-config (merge {"bootstrap.servers" brokers 175 | "group.id" (or group-id "onyx") 176 | "enable.auto.commit" false 177 | "receive.buffer.bytes" (or (:kafka/receive-buffer-bytes task-map) 178 | (:kafka/receive-buffer-bytes defaults)) 179 | "auto.offset.reset" (name (:kafka/offset-reset task-map))} 180 | consumer-opts) 181 | _ (info log-prefix "Starting kafka/read-messages task with consumer opts:" consumer-config) 182 | key-deserializer (h/byte-array-deserializer) 183 | value-deserializer (h/byte-array-deserializer) 184 | consumer* (h/build-consumer consumer-config key-deserializer value-deserializer) 185 | _ (when (and (:kafka/target-offsets task-map) 186 | (:kafka/start-offsets task-map) 187 | (not= (set (keys (:kafka/target-offsets task-map))) 188 | (set (keys (:kafka/start-offsets task-map))))) 189 | (throw (ex-info "When :kafka/start-offsets and :kafka/target-offsets are both specified, they must both contain the same partitions." 190 | task-map))) 191 | partitions (mapv :partition 192 | (or (keys (:kafka/target-offsets task-map)) 193 | (keys (:kafka/start-offsets task-map)) 194 | (h/partitions-for-topic consumer* topic))) 195 | n-partitions (count partitions)] 196 | (check-num-peers-equals-partitions task-map n-partitions) 197 | (let [kpartitions* (assign-partitions-to-slot! consumer* task-map topic n-partitions (:onyx.core/slot-id event))] 198 | (set! consumer consumer*) 199 | (set! kpartitions kpartitions*) 200 | this))) 201 | 202 | (stop [this event] 203 | (when consumer 204 | (.close consumer) 205 | (set! consumer nil)) 206 | this) 207 | 208 | p/WatermarkedInput 209 | (watermark [this] 210 | (.get watermark)) 211 | 212 | p/Checkpointed 213 | (checkpoint [this] 214 | partition->offset) 215 | ;; checkpoint map looks like {part offset} 216 | (recover! [this replica-version checkpoint] 217 | (let [partition-statuses (current-partition-statuses checkpoint consumer target-offsets kpartitions) 218 | resuming-tps (reduce-kv 219 | (fn [all part v] 220 | (if (not (some #{v} #{:emitted :drained})) 221 | (conj all (TopicPartition. topic part)) 222 | all)) 223 | [] 224 | partition-statuses)] 225 | (.resume consumer resuming-tps) 226 | (reset! drained partition-statuses) 227 | (set! iter nil) 228 | (set! partition->offset checkpoint) 229 | (seek-offset! log-prefix consumer kpartitions task-map topic checkpoint)) 230 | this) 231 | 232 | (checkpointed! [this epoch]) 233 | 234 | p/BarrierSynchronization 235 | (synced? [this epoch] 236 | (set-lag! lag-gauge consumer) 237 | true) 238 | 239 | (completed? [this] 240 | (empty? (remove #(= :emitted %) (vals @drained)))) 241 | 242 | p/Input 243 | (poll! [this _ remaining-ms] 244 | (if-let [drained-part (ffirst (filter (fn [[_ state]] (= state :drained)) @drained))] 245 | (do (swap! drained assoc drained-part :emitted) 246 | {:type :end-reached :partition drained-part}) 247 | (if (and iter (.hasNext ^java.util.Iterator iter)) 248 | (let [rec ^ConsumerRecord (.next ^java.util.Iterator iter) 249 | deserialized (some-> rec segment-fn) 250 | part (.partition rec)] 251 | (.set watermark (max (.get watermark) (.timestamp rec))) 252 | 253 | ;; TODO: remove support for :done in favor of target-offsets 254 | (cond (= :done deserialized) 255 | (do (swap! drained assoc (.partition rec) :emitted) 256 | nil) 257 | 258 | deserialized 259 | (let [new-offset (.offset rec)] 260 | (set! partition->offset (assoc partition->offset part new-offset)) 261 | (if-let [target-offset (get target-offsets part)] 262 | (let [tp (TopicPartition. topic part)] 263 | (if (>= new-offset target-offset) 264 | (when (not (paused? consumer part)) 265 | (.pause consumer [tp]) 266 | (swap! drained assoc (.partition rec) :drained) 267 | ;; only emit message if it's on the boundary 268 | (if (= new-offset target-offset) 269 | deserialized)) 270 | deserialized)) 271 | deserialized)))) 272 | (do (set! iter (.iterator ^ConsumerRecords (.poll ^Consumer consumer remaining-ms))) 273 | nil))))) 274 | 275 | (defn read-messages [{:keys [onyx.core/task-map onyx.core/log-prefix onyx.core/monitoring]}] 276 | (let [{:keys [kafka/topic kafka/deserializer-fn kafka/target-offsets]} task-map 277 | batch-timeout (arg-or-default :onyx/batch-timeout task-map) 278 | wrap-message? (or (:kafka/wrap-with-metadata? task-map) (:kafka/wrap-with-metadata? defaults)) 279 | deserializer-fn (kw->fn (:kafka/deserializer-fn task-map)) 280 | key-deserializer-fn (if-let [kw (:kafka/key-deserializer-fn task-map)] (kw->fn kw) identity) 281 | segment-fn (if wrap-message? 282 | (fn [^ConsumerRecord cr] 283 | {:topic (.topic cr) 284 | :partition (.partition cr) 285 | :key (when-let [k (.key cr)] (key-deserializer-fn k)) 286 | :message (deserializer-fn (.value cr)) 287 | :serialized-key-size (.serializedKeySize cr) 288 | :serialized-value-size (.serializedValueSize cr) 289 | :timestamp (.timestamp cr) 290 | :timestamp-type (.id ^TimestampType (.timestampType cr)) 291 | :offset (.offset cr)}) 292 | (fn [^ConsumerRecord cr] 293 | (deserializer-fn (.value cr)))) 294 | watermark (AtomicLong. 0) 295 | {:keys [lag-gauge]} monitoring] 296 | (->KafkaReadMessages log-prefix task-map topic nil batch-timeout 297 | deserializer-fn segment-fn watermark lag-gauge 298 | nil nil nil (atom {}) target-offsets))) 299 | 300 | (defn close-read-messages 301 | [event lifecycle] 302 | {}) 303 | 304 | (defn inject-write-messages 305 | [event lifecycle] 306 | {}) 307 | 308 | (defn close-write-resources 309 | [event lifecycle] 310 | {}) 311 | 312 | (defn- message->producer-record 313 | [key-serializer-fn serializer-fn topic kpartition m] 314 | (let [message (:message m) 315 | k (some-> m :key key-serializer-fn) 316 | message-topic (get m :topic topic) 317 | message-partition (some-> m (get :partition kpartition) int) 318 | message-timestamp (some-> m (get :timestamp) long)] 319 | (cond (not (contains? m :message)) 320 | (throw (ex-info "Payload is missing required. Need message key :message" 321 | {:recoverable? false 322 | :payload m})) 323 | 324 | (nil? message-topic) 325 | (throw (ex-info 326 | (str "Unable to write message payload to Kafka! " 327 | "Both :kafka/topic, and :topic in message payload " 328 | "are missing!") 329 | {:recoverable? false 330 | :payload m})) 331 | 332 | :else 333 | (ProducerRecord. ^String message-topic ^Integer message-partition ^Long message-timestamp k (serializer-fn message))))) 334 | 335 | (defn clear-write-futures! [fs] 336 | (doall (remove (fn [^java.util.concurrent.Future f] 337 | (assert (not (.isCancelled f))) 338 | (.isDone f)) 339 | fs))) 340 | 341 | (defrecord KafkaWriteMessages [task-map config topic kpartition producer key-serializer-fn serializer-fn write-futures exception write-callback] 342 | p/Plugin 343 | (start [this event] 344 | this) 345 | 346 | (stop [this event] 347 | (.close ^KafkaProducer producer) 348 | this) 349 | 350 | p/BarrierSynchronization 351 | (synced? [this epoch] 352 | (when @exception (throw @exception)) 353 | (empty? (vswap! write-futures clear-write-futures!))) 354 | 355 | (completed? [this] 356 | (when @exception (throw @exception)) 357 | (empty? (vswap! write-futures clear-write-futures!))) 358 | 359 | p/Checkpointed 360 | (recover! [this _ _] 361 | this) 362 | 363 | (checkpoint [this]) 364 | 365 | (checkpointed! [this epoch]) 366 | 367 | p/Output 368 | (prepare-batch [this event replica _] 369 | true) 370 | 371 | (write-batch [this {:keys [onyx.core/write-batch]} replica _] 372 | (when @exception (throw @exception)) 373 | (vswap! write-futures 374 | (fn [fs] 375 | (-> fs 376 | (clear-write-futures!) 377 | (into (map 378 | (fn [msg] 379 | (let [record (message->producer-record key-serializer-fn serializer-fn topic kpartition msg)] 380 | (.send ^KafkaProducer producer record write-callback))) 381 | write-batch))))) 382 | true)) 383 | 384 | (def write-defaults {:kafka/request-size 307200}) 385 | 386 | (deftype ExceptionCallback [e] 387 | Callback 388 | (onCompletion [_ v exception] 389 | (when exception (reset! e exception)))) 390 | 391 | (defn write-messages [{:keys [onyx.core/task-map onyx.core/log-prefix]}] 392 | (let [_ (s/validate onyx.tasks.kafka/KafkaOutputTaskMap task-map) 393 | brokers (or (:kafka/bootstrap-servers task-map) (find-brokers task-map)) 394 | request-size (or (get task-map :kafka/request-size) (get write-defaults :kafka/request-size)) 395 | producer-opts (:kafka/producer-opts task-map) 396 | config (merge {"bootstrap.servers" brokers 397 | "max.request.size" request-size} 398 | producer-opts) 399 | _ (info log-prefix "Starting kafka/write-messages task with producer opts:" config) 400 | topic (:kafka/topic task-map) 401 | kpartition (:kafka/partition task-map) 402 | key-serializer (h/byte-array-serializer) 403 | value-serializer (h/byte-array-serializer) 404 | producer (h/build-producer config key-serializer value-serializer) 405 | serializer-fn (kw->fn (:kafka/serializer-fn task-map)) 406 | key-serializer-fn (if-let [kw (:kafka/key-serializer-fn task-map)] (kw->fn kw) identity) 407 | exception (atom nil) 408 | write-callback (->ExceptionCallback exception) 409 | write-futures (volatile! (list))] 410 | (->KafkaWriteMessages task-map config topic kpartition producer 411 | key-serializer-fn serializer-fn 412 | write-futures exception write-callback))) 413 | 414 | (defn read-handle-exception [event lifecycle lf-kw exception] 415 | (if (false? (:recoverable? (ex-data exception))) 416 | :kill 417 | :restart)) 418 | 419 | (def read-messages-calls 420 | {:lifecycle/before-task-start start-kafka-consumer 421 | :lifecycle/handle-exception read-handle-exception 422 | :lifecycle/after-task-stop close-read-messages}) 423 | 424 | (defn write-handle-exception [event lifecycle lf-kw exception] 425 | (if (false? (:recoverable? (ex-data exception))) 426 | :kill 427 | :restart)) 428 | 429 | (def write-messages-calls 430 | {:lifecycle/before-task-start inject-write-messages 431 | :lifecycle/handle-exception write-handle-exception 432 | :lifecycle/after-task-stop close-write-resources}) 433 | -------------------------------------------------------------------------------- /src/onyx/plugin/partition_assignment.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.partition-assignment) 2 | 3 | ;; Given a number of partitions, a number of peers, and slow for a single 4 | ;; peer, calculcates which partitions this slot should be assigned. 5 | ;; Disperses the peers over the partitions such that 6 | ;; the assignments are as evenly distributed as possible. 7 | ;; 8 | ;; First, we calculate the minimum number of partitions that 9 | ;; all peers will receive by doing division and flooring the answer. 10 | ;; Next, we take the mod of the partitions the peers, because flooring 11 | ;; the division result may have resulted in a certain number of unassigned 12 | ;; partitions. A strict subset of the peers may receive at most one more 13 | ;; partition. If a peer's slot, which is a zero-based index, is less than 14 | ;; or equal to the number of left over partitions, it can bump it's count by 15 | ;; 1 to receive one of the extras - otherwise the count remains the same. 16 | ;; 17 | ;; Now that we know how many partitions this peer's slot should receive, 18 | ;; we need to know exactly which partitions those are. Remember that partitions 19 | ;; are also zero-indexed. 20 | ;; 21 | ;; Calculating the upper-bound is easy - we take the number of partitions 22 | ;; and add it to the lower bound, decrementing by 1 to account for the zero 23 | ;; index. Thus, the only remaining variable is the lower bound. 24 | ;; 25 | ;; The lower bound is calculated by summing up all the partitions for all 26 | ;; the slots before it. We know that there are at least the base number 27 | ;; of partitions (via floored division) multiplied by this slot assigned 28 | ;; before this slot. We then need to account for the extras by adding the 29 | ;; minimum of how many partitions are left over and this slot index. 30 | ;; 31 | ;; Example: 11 partitions, 3 slots 32 | ;; 33 | ;; Slot Partitions 34 | ;; 0 0, 1, 2, 3 35 | ;; 1 4, 5, 6, 7 36 | ;; 2 8, 9, 10 37 | ;; 38 | ;; Example: 7 partitions, 3 slots 39 | ;; 40 | ;; Slot Partitions 41 | ;; 0 0, 1, 2 42 | ;; 1 3, 4 43 | ;; 2 5, 6 44 | ;; 45 | (defn partitions-for-slot [n-partitions n-peers my-slot] 46 | (when (> n-peers n-partitions) 47 | (throw (ex-info "Number of peers assigned to this task exceeds the number of partitions in the Topic. It must be less than or equal to it." 48 | {:n-partitions n-partitions 49 | :n-peers n-peers}))) 50 | (let [at-least (long (/ n-partitions n-peers)) 51 | left-over (mod n-partitions n-peers) 52 | my-extra (if (<= (inc my-slot) left-over) 1 0) 53 | my-n-partitions (+ at-least my-extra) 54 | lower (+ (* at-least my-slot) (min left-over my-slot))] 55 | [lower (+ lower (dec my-n-partitions))])) 56 | -------------------------------------------------------------------------------- /src/onyx/tasks/kafka.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.tasks.kafka 2 | (:require [cheshire.core :as json] 3 | [schema.core :as s] 4 | [onyx.job :refer [add-task]] 5 | [onyx.schema :as os]) 6 | (:import [com.fasterxml.jackson.core JsonGenerationException])) 7 | 8 | ;;;; Reader task 9 | (defn deserialize-message-json [^bytes bs] 10 | (try 11 | (json/parse-string (String. bs "UTF-8")) 12 | (catch Exception e 13 | {:error e}))) 14 | 15 | (defn deserialize-message-edn [^bytes bs] 16 | (try 17 | (read-string (String. bs "UTF-8")) 18 | (catch Exception e 19 | {:error e}))) 20 | 21 | (def KafkaInputTaskMap 22 | {:kafka/topic s/Str 23 | :kafka/offset-reset (s/enum :earliest :latest) 24 | :kafka/deserializer-fn os/NamespacedKeyword 25 | (s/optional-key :kafka/bootstrap-servers) [s/Str] 26 | (s/optional-key :kafka/zookeeper) s/Str 27 | (s/optional-key :kafka/key-deserializer-fn) os/NamespacedKeyword 28 | (s/optional-key :kafka/group-id) s/Str 29 | (s/optional-key :kafka/consumer-opts) {s/Any s/Any} 30 | (s/optional-key :kafka/start-offsets) {s/Int s/Int} 31 | (s/optional-key :kafka/receive-buffer-bytes) s/Int 32 | (s/optional-key :kafka/partition) (s/cond-pre s/Int s/Str) 33 | (s/optional-key :kafka/wrap-with-metadata?) s/Bool 34 | (s/optional-key :kafka/target-offsets) {s/Int s/Int} 35 | (os/restricted-ns :kafka) s/Any}) 36 | 37 | (s/defn ^:always-validate consumer 38 | ([task-name :- s/Keyword opts] 39 | {:task {:task-map (merge {:onyx/name task-name 40 | :onyx/plugin :onyx.plugin.kafka/read-messages 41 | :onyx/type :input 42 | :onyx/medium :kafka 43 | :kafka/receive-buffer-bytes 65536 44 | :kafka/wrap-with-metadata? false 45 | :onyx/doc "Reads messages from a Kafka topic"} 46 | opts) 47 | :lifecycles [{:lifecycle/task task-name 48 | :lifecycle/calls :onyx.plugin.kafka/read-messages-calls}]} 49 | :schema {:task-map KafkaInputTaskMap}}) 50 | ([task-name :- s/Keyword 51 | topic :- s/Str 52 | group-id :- s/Str 53 | zookeeper :- s/Str 54 | offset-reset :- (s/enum :earliest :latest) 55 | deserializer-fn :- os/NamespacedKeyword 56 | task-opts :- {s/Any s/Any}] 57 | (consumer task-name (merge {:kafka/topic topic 58 | :kafka/group-id group-id 59 | :kafka/zookeeper zookeeper 60 | :kafka/offset-reset offset-reset 61 | :kafka/deserializer-fn deserializer-fn} 62 | task-opts)))) 63 | 64 | ;;;; Writer task 65 | (defn serialize-message-json [segment] 66 | (try 67 | (.getBytes (json/generate-string segment)) 68 | (catch JsonGenerationException e 69 | (throw (ex-info (format "Could not serialize segment: %s" segment) 70 | {:recoverable? false 71 | :segment segment 72 | :cause e}))))) 73 | 74 | (defn serialize-message-edn [segment] 75 | (.getBytes (pr-str segment))) 76 | 77 | (def KafkaOutputTaskMap 78 | {(s/optional-key :kafka/topic) s/Str 79 | :kafka/serializer-fn os/NamespacedKeyword 80 | (s/optional-key :kafka/bootstrap-servers) [s/Str] 81 | (s/optional-key :kafka/zookeeper) s/Str 82 | (s/optional-key :kafka/key-serializer-fn) os/NamespacedKeyword 83 | (s/optional-key :kafka/request-size) s/Num 84 | (s/optional-key :kafka/partition) (s/cond-pre s/Int s/Str) 85 | (s/optional-key :kafka/no-seal?) s/Bool 86 | (s/optional-key :kafka/producer-opts) {s/Any s/Any} 87 | (os/restricted-ns :kafka) s/Any}) 88 | 89 | (s/defn ^:always-validate producer 90 | ([task-name :- s/Keyword opts] 91 | {:task {:task-map (merge {:onyx/name task-name 92 | :onyx/plugin :onyx.plugin.kafka/write-messages 93 | :onyx/type :output 94 | :onyx/medium :kafka 95 | :onyx/doc "Writes messages to a kafka topic"} 96 | opts) 97 | :lifecycles [{:lifecycle/task task-name 98 | :lifecycle/calls :onyx.plugin.kafka/write-messages-calls}]} 99 | :schema {:task-map KafkaOutputTaskMap}}) 100 | ([task-name :- s/Keyword 101 | topic :- s/Str 102 | zookeeper :- s/Str 103 | serializer-fn :- os/NamespacedKeyword 104 | request-size :- s/Num 105 | task-opts :- {s/Any s/Any}] 106 | (producer task-name (merge {:kafka/topic topic 107 | :kafka/zookeeper zookeeper 108 | :kafka/serializer-fn serializer-fn 109 | :kafka/request-size request-size} 110 | task-opts)))) 111 | -------------------------------------------------------------------------------- /test-resources/config.edn: -------------------------------------------------------------------------------- 1 | {:test-config 2 | {:kafka-bootstrap #env [KAFKA_BOOTSTRAP ["127.0.0.1:9092"]] 3 | :embedded-kafka? #cond {:default false 4 | :test false 5 | :ci false}} 6 | :env-config 7 | {:onyx/tenancy-id #env [ONYX_ID "testcluster"] 8 | :zookeeper/address #cond {:default #env [ZOOKEEPER "zookeeper"] 9 | :bench "127.0.0.1:2181" 10 | :ci "127.0.0.1:2181" 11 | :test "127.0.0.1:2181"} 12 | :zookeeper/server? #cond {:default false 13 | :ci false 14 | :test false} 15 | :zookeeper.server/port 2181} 16 | :peer-config 17 | {:onyx/tenancy-id #env [ONYX_ID "testcluster"] 18 | :zookeeper/address #cond {:default #env [ZOOKEEPER "zookeeper"] 19 | :bench "127.0.0.1:2181" 20 | :ci "127.0.0.1:2181" 21 | :test #env [ZOOKEEPER "127.0.0.1:2181"]} 22 | :onyx.peer/job-scheduler :onyx.job-scheduler/greedy 23 | :onyx.peer/zookeeper-timeout 60000 24 | :onyx.peer/storage.zk.insanely-allow-windowing? true 25 | :onyx.messaging/allow-short-circuit? #cond {:default true 26 | :bench true 27 | :test true} 28 | :onyx.messaging/impl :aeron 29 | :onyx.messaging/bind-addr #env [BIND_ADDR "localhost"] 30 | :onyx.messaging/peer-port 40200 31 | :onyx.messaging.aeron/embedded-driver? #cond {:default true 32 | :bench true 33 | :ci true 34 | :test true}}} 35 | -------------------------------------------------------------------------------- /test/onyx/plugin/doc_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.doc-test 2 | (:require [taoensso.timbre :refer [info] :as timbre] 3 | [clojure.test :refer [deftest is testing]] 4 | [onyx.kafka.information-model :refer [model]])) 5 | 6 | (deftest check-model-display-order 7 | (testing "Checks whether all keys in information model are accounted for in ordering used in cheat sheet" 8 | (let [{:keys [catalog-entry display-order]} model] 9 | (doall 10 | (for [[task-type task-info] catalog-entry] 11 | (let [display-order (get display-order task-type)] 12 | (is (= (set display-order) 13 | (set (keys (:model task-info))))))))))) 14 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_benchmark_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-benchmark-test 2 | (:require [clojure.core.async :refer [ base-job 57 | (add-task (consumer :read-messages 58 | {:kafka/topic topic 59 | :kafka/group-id "onyx-consumer-1" 60 | :kafka/zookeeper zk-address 61 | :kafka/offset-reset :earliest 62 | :kafka/receive-buffer-bytes 65536 63 | :kafka/deserializer-fn ::decompress 64 | :onyx/fn ::print-message 65 | :onyx/batch-timeout 50 66 | :onyx/batch-size batch-size 67 | :onyx/min-peers n-partitions 68 | :onyx/max-peers n-partitions})) 69 | (add-task (core-async/output :out 70 | {:onyx/batch-timeout batch-timeout 71 | :onyx/batch-size batch-size} 72 | 100000000))))) 73 | 74 | (defn write-data 75 | [topic zookeeper bootstrap-servers] 76 | (h/create-topic! zookeeper topic n-partitions 1) 77 | (let [producer-config {"bootstrap.servers" bootstrap-servers} 78 | key-serializer (h/byte-array-serializer) 79 | value-serializer (h/byte-array-serializer) 80 | producer1 (h/build-producer producer-config key-serializer value-serializer)] 81 | (time 82 | (doseq [p (range n-partitions)] 83 | (mapv deref 84 | (doall 85 | (map (fn [x] 86 | ;; 116 bytes messages 87 | (.send producer1 (ProducerRecord. topic 88 | (int p) 89 | nil 90 | (compress {:n x :really-long-string (apply str (repeatedly 30 (fn [] (rand-int 500))))})))) 91 | (range messages-per-partition)))))) 92 | (println "Successfully wrote messages"))) 93 | 94 | (defn take-until-nothing! 95 | [ch timeout-ms] 96 | (loop [ret []] 97 | (let [tmt (if timeout-ms (timeout timeout-ms) (chan)) 98 | [v c] (alts!! [ch tmt] :priority true)] 99 | (if (= c tmt) 100 | ret 101 | (if (and v (not= v :done)) 102 | (recur (conj ret v)) 103 | (conj ret :done)))))) 104 | 105 | (deftest ^:benchmark kafka-input-test 106 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 107 | {:keys [env-config peer-config test-config]} (read-config (clojure.java.io/resource "config.edn") 108 | {:profile :bench}) 109 | tenancy-id (str (java.util.UUID/randomUUID)) 110 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 111 | peer-group (onyx.api/start-peer-group peer-config) 112 | n-peers (+ 2 n-partitions) 113 | v-peers (onyx.api/start-peers n-peers peer-group) 114 | zk-address (get-in peer-config [:zookeeper/address]) 115 | job (build-job zk-address test-topic 100 50) 116 | {:keys [out read-messages]} (get-core-async-channels job)] 117 | (try 118 | (println "Topic is " test-topic) 119 | (write-data test-topic zk-address (:kafka-bootstrap test-config)) 120 | ;; Appropriate time to settle before submitting the job 121 | (Thread/sleep 5000) 122 | (let [job-ret (onyx.api/submit-job peer-config job) 123 | _ (println "Job ret" job-ret) 124 | job-id (:job-id job-ret) 125 | start-time (System/currentTimeMillis) 126 | read-nothing-timeout 30000 127 | read-segments (take-until-nothing! out read-nothing-timeout)] 128 | (is (= (* n-partitions messages-per-partition) (count read-segments))) 129 | (let [run-time (- (System/currentTimeMillis) start-time read-nothing-timeout) 130 | n-messages-total (* n-partitions messages-per-partition)] 131 | (println (float (* 1000 (/ n-messages-total run-time))) 132 | "messages per second. Processed" n-messages-total 133 | "messages in" run-time "ms.")) 134 | (onyx.api/kill-job peer-config job-id)) 135 | (finally 136 | (doseq [p v-peers] 137 | (onyx.api/shutdown-peer p)) 138 | (onyx.api/shutdown-peer-group peer-group))))) 139 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_broker_reboot_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-broker-reboot-test 2 | (:require [aero.core :refer [read-config]] 3 | [clojure.core.async :refer [>!! chan]] 4 | [onyx.kafka.helpers :as h] 5 | [clojure.test :refer [deftest is]] 6 | [com.stuartsierra.component :as component] 7 | [onyx api 8 | [job :refer [add-task]] 9 | [test-helper :refer [with-test-env]]] 10 | [clojure.java.shell :refer [sh]] 11 | [onyx.plugin kafka 12 | [core-async :refer [take-segments! get-core-async-channels]] 13 | [test-utils :as test-utils]] 14 | [onyx.tasks 15 | [kafka :refer [consumer]] 16 | [core-async :as core-async]])) 17 | 18 | (defn build-job [zk-address topic batch-size batch-timeout] 19 | (let [batch-settings {:onyx/batch-size batch-size :onyx/batch-timeout batch-timeout} 20 | base-job (merge {:workflow [[:read-messages :identity] 21 | [:identity :out]] 22 | :catalog [(merge {:onyx/name :identity 23 | :onyx/fn :clojure.core/identity 24 | :onyx/type :function} 25 | batch-settings)] 26 | :lifecycles [{:lifecycle/task :all 27 | :lifecycle/calls ::restartable-reader}] 28 | :windows [] 29 | :triggers [] 30 | :flow-conditions [] 31 | :task-scheduler :onyx.task-scheduler/balanced})] 32 | (-> base-job 33 | (add-task (consumer :read-messages 34 | (merge {:kafka/topic topic 35 | :kafka/group-id "onyx-consumer" 36 | :kafka/zookeeper zk-address 37 | :kafka/offset-reset :earliest 38 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 39 | :onyx/n-peers 1 40 | :onyx/batch-size 2} 41 | batch-settings))) 42 | (add-task (core-async/output :out batch-settings))))) 43 | 44 | (defn stop-kafka [mock embedded-kafka?] 45 | (if embedded-kafka? 46 | (swap! mock component/stop) 47 | (do 48 | (println "Stopping Docker Kafka instance") 49 | (sh "docker" "stop" "onyxkafka_kafka_1")))) 50 | 51 | (defn start-kafka [mock embedded-kafka?] 52 | (if embedded-kafka? 53 | (swap! mock component/start) 54 | (do 55 | (println "Starting Docker Kafka instance") 56 | (sh "docker" "start" "onyxkafka_kafka_1")))) 57 | 58 | (def restartable-reader 59 | {:lifecycle/handle-exception (constantly :restart)}) 60 | 61 | (deftest ^:broker-reboot kafka-broker-reboot-test 62 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 63 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 64 | embedded-kafka? (:embedded-kafka? test-config) 65 | tenancy-id (str (java.util.UUID/randomUUID)) 66 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 67 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 68 | zk-address (get-in peer-config [:zookeeper/address]) 69 | job (build-job zk-address test-topic 2 1000) 70 | {:keys [out read-messages]} (get-core-async-channels job) 71 | test-data1 (mapv (fn [i] {:n i}) (range 0 100)) 72 | test-data2 (mapv (fn [i] {:n i}) (range 101 200)) 73 | mock (atom {}) 74 | _ (h/create-topic! zk-address test-topic 2 1)] 75 | (with-test-env [test-env [4 env-config peer-config]] 76 | (onyx.test-helper/validate-enough-peers! test-env job) 77 | (test-utils/write-data test-topic zk-address (:kafka-bootstrap test-config) test-data1) 78 | (let [job-id (:job-id (onyx.api/submit-job peer-config job))] 79 | (Thread/sleep 10000) 80 | (stop-kafka mock embedded-kafka?) 81 | (Thread/sleep 20000) 82 | (start-kafka mock embedded-kafka?) 83 | ;; wait for long enough before putting onto the input channel 84 | ;; otherwise it'll try to write to kafka before it's back up 85 | (Thread/sleep 60000) 86 | (test-utils/write-data test-topic zk-address (:kafka-bootstrap test-config) test-data2) 87 | ;(onyx.test-helper/feedback-exception! peer-config job-id) 88 | (is (= (set (into test-data1 test-data2)) 89 | (set (onyx.plugin.core-async/take-segments! out 20000)))))))) 90 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_multipartition_scheduling_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-multipartition-scheduling-test 2 | (:require [clojure.test :refer [deftest is]] 3 | [com.stuartsierra.component :as component] 4 | [onyx.test-helper :refer [with-test-env]] 5 | [onyx.job :refer [add-task]] 6 | [onyx.tasks.kafka :refer [consumer]] 7 | [onyx.tasks.core-async :as core-async] 8 | [onyx.kafka.helpers :as h] 9 | [onyx.plugin.test-utils :as test-utils] 10 | [onyx.plugin.core-async :refer [get-core-async-channels]] 11 | [onyx.plugin.kafka] 12 | [onyx.api])) 13 | 14 | (def n-partitions 4) 15 | 16 | (defn build-job [zk-address topic batch-size batch-timeout n-input-peers] 17 | (let [batch-settings {:onyx/batch-size batch-size :onyx/batch-timeout batch-timeout} 18 | base-job (merge {:workflow [[:read-messages :identity] 19 | [:identity :out]] 20 | :catalog [(merge {:onyx/name :identity 21 | :onyx/fn :clojure.core/identity 22 | :onyx/type :function} 23 | batch-settings)] 24 | :lifecycles [] 25 | :windows [] 26 | :triggers [] 27 | :flow-conditions [] 28 | :task-scheduler :onyx.task-scheduler/balanced})] 29 | (-> base-job 30 | (add-task (consumer :read-messages 31 | (merge {:kafka/topic topic 32 | :kafka/group-id "onyx-consumer" 33 | :kafka/zookeeper zk-address 34 | :kafka/offset-reset :earliest 35 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 36 | :onyx/n-peers n-partitions} 37 | batch-settings))) 38 | (add-task (core-async/output :out batch-settings))))) 39 | 40 | (defn write-data 41 | [topic zookeeper kafka-bootstrap n-segments-each] 42 | (h/create-topic! zookeeper topic n-partitions 1) 43 | (let [producer-config {"bootstrap.servers" kafka-bootstrap} 44 | key-serializer (h/byte-array-serializer) 45 | value-serializer (h/byte-array-serializer)] 46 | (with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)] 47 | (with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)] 48 | (doseq [x (range n-segments-each)] ;0 1 2 49 | (h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x})))) 50 | (doseq [x (range n-segments-each)] ;3 4 5 51 | (h/send-sync! producer2 topic nil nil (.getBytes (pr-str {:n (+ n-segments-each x)})))))))) 52 | 53 | (deftest kafka-multipartition-scheduling-test 54 | (let [test-topic (str (java.util.UUID/randomUUID)) 55 | _ (println "Using topic" test-topic) 56 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 57 | kafka-bootstrap (:kafka-bootstrap test-config) 58 | tenancy-id (str (java.util.UUID/randomUUID)) 59 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 60 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 61 | zk-address (get-in peer-config [:zookeeper/address]) 62 | ;; randomize number of peers so that we can check if number of partitions 63 | ;; is correctly assigned 64 | n-peers (inc (rand-int n-partitions)) 65 | _ (println "Random number of npeers" n-peers) 66 | job (build-job zk-address test-topic 10 1000 n-peers) 67 | {:keys [out read-messages]} (get-core-async-channels job) 68 | n-segments-each 200] 69 | (with-test-env [test-env [(+ n-partitions 2) env-config peer-config]] 70 | (onyx.test-helper/validate-enough-peers! test-env job) 71 | (write-data test-topic zk-address kafka-bootstrap n-segments-each) 72 | (let [job-id (:job-id (onyx.api/submit-job peer-config job))] 73 | (println "Taking segments") 74 | ;(onyx.test-helper/feedback-exception! peer-config job-id) 75 | (let [results (onyx.plugin.core-async/take-segments! out 10000)] 76 | (is (= (range (* 2 n-segments-each)) (sort (mapv :n results))))) 77 | (println "Done taking segments") 78 | (onyx.api/kill-job peer-config job-id))))) 79 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_resume_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-resume-test 2 | (:require [aero.core :refer [read-config]] 3 | [clojure.test :refer [deftest is]] 4 | [com.stuartsierra.component :as component] 5 | [onyx api 6 | [job :refer [add-task]] 7 | [test-helper :refer [with-test-env]]] 8 | [onyx.plugin kafka 9 | [core-async :refer [take-segments! get-core-async-channels]] 10 | [test-utils :as test-utils]] 11 | [onyx.tasks 12 | [kafka :refer [consumer]] 13 | [core-async :as core-async]])) 14 | 15 | (defn build-job [zk-address topic batch-size batch-timeout] 16 | (let [batch-settings {:onyx/batch-size batch-size 17 | :onyx/batch-timeout batch-timeout} 18 | base-job (merge {:workflow [[:read-messages :identity] 19 | [:identity :out]] 20 | :catalog [(merge {:onyx/name :identity 21 | :onyx/fn :clojure.core/identity 22 | :onyx/n-peers 1 23 | :onyx/type :function} 24 | batch-settings)] 25 | :lifecycles [{:lifecycle/task :read-messages 26 | :lifecycle/calls ::read-crash}] 27 | :windows [{:window/id :collect-segments 28 | :window/task :identity 29 | :window/type :global 30 | :window/aggregation :onyx.windowing.aggregation/conj}] 31 | :triggers [{:trigger/window-id :collect-segments 32 | :trigger/fire-all-extents? true 33 | :trigger/id :collect-trigger 34 | :trigger/on :onyx.triggers/segment 35 | :trigger/threshold [1 :elements] 36 | :trigger/sync ::update-atom!}] 37 | :flow-conditions [] 38 | :task-scheduler :onyx.task-scheduler/balanced})] 39 | (-> base-job 40 | (add-task (consumer :read-messages 41 | (merge {:kafka/topic topic 42 | :kafka/group-id "onyx-consumer" 43 | :kafka/zookeeper zk-address 44 | :kafka/offset-reset :earliest 45 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 46 | :onyx/max-peers 1 47 | :onyx/batch-size 1} 48 | batch-settings))) 49 | (add-task (core-async/output :out batch-settings 100000))))) 50 | 51 | (def batch-num (atom 0)) 52 | 53 | (def test-state (atom #{})) 54 | 55 | (defn update-atom! [event window trigger {:keys [lower-bound upper-bound event-type] :as state-event} extent-state] 56 | (swap! test-state into extent-state)) 57 | 58 | (def read-crash 59 | {:lifecycle/before-batch 60 | (fn [event lifecycle] 61 | (when (= (swap! batch-num inc) 4000) 62 | (throw (ex-info "Restartable" {:restartable? true})))) 63 | :lifecycle/handle-exception (constantly :restart)}) 64 | 65 | (deftest kafka-resume-test 66 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 67 | _ (println "Using topic" test-topic) 68 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 69 | tenancy-id (str (java.util.UUID/randomUUID)) 70 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 71 | peer-config (assoc peer-config 72 | :onyx/tenancy-id tenancy-id 73 | ;; should be much lower to get some checkpointing in 74 | :onyx.peer/coordinator-barrier-period-ms 1) 75 | zk-address (get-in peer-config [:zookeeper/address]) 76 | job (build-job zk-address test-topic 2 1000) 77 | test-data (conj (mapv (fn [v] {:n v}) (range 10000)) :done)] 78 | (with-test-env [test-env [4 env-config peer-config]] 79 | (onyx.test-helper/validate-enough-peers! test-env job) 80 | (test-utils/write-data test-topic zk-address (:kafka-bootstrap test-config) test-data) 81 | (->> job 82 | (onyx.api/submit-job peer-config) 83 | :job-id 84 | (onyx.test-helper/feedback-exception! peer-config)) 85 | (Thread/sleep 1000) 86 | (let [{:keys [out]} (get-core-async-channels job)] 87 | (is (= (set (butlast test-data)) (set @test-state))))))) 88 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_seek_latest_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-seek-latest-test 2 | (:require [clojure.core.async :refer [ base-job 30 | (add-task (consumer :read-messages 31 | (merge {:kafka/topic topic 32 | :kafka/group-id "onyx-consumer" 33 | :kafka/zookeeper zk-address 34 | :kafka/offset-reset :latest 35 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 36 | :onyx/min-peers 2 37 | :onyx/max-peers 2} 38 | batch-settings))) 39 | (add-task (core-async/output :out batch-settings))))) 40 | 41 | (defn write-messages 42 | "Use a custom version of mock-kafka as opposed to the one in test-utils 43 | because we need to spawn 2 producers in order to write to each partition" 44 | [topic zookeeper bootstrap-servers] 45 | (let [producer-config {"bootstrap.servers" bootstrap-servers} 46 | key-serializer (h/byte-array-serializer) 47 | value-serializer (h/byte-array-serializer)] 48 | (with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)] 49 | (with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)] 50 | (doseq [x (range 3)] ;0 1 2 51 | (h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x})))) 52 | (doseq [x (range 3)] ;3 4 5 53 | (h/send-sync! producer2 topic nil nil (.getBytes (pr-str {:n (+ 3 x)})))))))) 54 | 55 | (deftest kafka-input-test 56 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 57 | _ (println "Using topic" test-topic) 58 | {:keys [env-config test-config peer-config]} (read-config (clojure.java.io/resource "config.edn") 59 | {:profile :test}) 60 | tenancy-id (str (java.util.UUID/randomUUID)) 61 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 62 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 63 | zk-address (get-in peer-config [:zookeeper/address]) 64 | job (build-job zk-address test-topic 10 1000) 65 | {:keys [out read-messages]} (get-core-async-channels job)] 66 | (with-test-env [test-env [4 env-config peer-config]] 67 | (onyx.test-helper/validate-enough-peers! test-env job) 68 | (h/create-topic! zk-address test-topic 2 1) 69 | (write-messages test-topic zk-address (:kafka-bootstrap test-config)) 70 | (let [job-id (:job-id (onyx.api/submit-job peer-config job))] 71 | (Thread/sleep 2000) 72 | (write-messages test-topic zk-address (:kafka-bootstrap test-config)) 73 | (is (= 15 (reduce + (mapv :n (onyx.plugin.core-async/take-segments! out 10000))))) 74 | (onyx.api/kill-job peer-config job-id) 75 | (Thread/sleep 10000))))) 76 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_start_offset_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-start-offset-test 2 | (:require [clojure.core.async :refer [ base-job 31 | (add-task (consumer :read-messages 32 | (merge {:kafka/topic topic 33 | :kafka/group-id "onyx-consumer" 34 | :kafka/zookeeper zk-address 35 | :kafka/offset-reset :earliest 36 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 37 | :kafka/start-offsets {0 1} 38 | :onyx/min-peers 1 39 | :onyx/max-peers 1} 40 | batch-settings))) 41 | (add-task (core-async/output :out batch-settings))))) 42 | 43 | (defn write-data-out 44 | [topic zookeeper bootstrap-servers] 45 | (h/create-topic! zookeeper topic 1 1) 46 | (let [producer-config {"bootstrap.servers" bootstrap-servers} 47 | key-serializer (h/byte-array-serializer) 48 | value-serializer (h/byte-array-serializer)] 49 | (with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)] 50 | (doseq [x (range 5)] ;0 1 2 51 | (Thread/sleep 500) 52 | (h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x}))))))) 53 | 54 | (deftest kafka-input-start-offset-test 55 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 56 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 57 | tenancy-id (str (java.util.UUID/randomUUID)) 58 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 59 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 60 | zk-address (get-in peer-config [:zookeeper/address]) 61 | job (build-job zk-address test-topic 10 1000) 62 | {:keys [out read-messages]} (get-core-async-channels job)] 63 | (with-test-env [test-env [4 env-config peer-config]] 64 | (onyx.test-helper/validate-enough-peers! test-env job) 65 | (write-data-out test-topic zk-address (:kafka-bootstrap test-config)) 66 | (->> job 67 | (onyx.api/submit-job peer-config) 68 | (:job-id)) 69 | (Thread/sleep 10000) 70 | (is (= [{:n 1} {:n 2} {:n 3} {:n 4}] (onyx.plugin.core-async/take-segments! out 10)))))) 71 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_static_partition_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-static-partition-test 2 | (:require [aero.core :refer [read-config]] 3 | [clojure.test :refer [deftest is]] 4 | [com.stuartsierra.component :as component] 5 | [onyx api 6 | [job :refer [add-task]] 7 | [test-helper :refer [with-test-env]]] 8 | [onyx.plugin kafka 9 | [core-async :refer [take-segments! get-core-async-channels]] 10 | [test-utils :as test-utils]] 11 | [onyx.tasks 12 | [kafka :refer [consumer]] 13 | [core-async :as core-async]])) 14 | 15 | (defn build-job [zk-address topic batch-size batch-timeout] 16 | (let [batch-settings {:onyx/batch-size batch-size :onyx/batch-timeout batch-timeout} 17 | base-job (merge {:workflow [[:read-messages :identity] 18 | [:identity :out]] 19 | :catalog [(merge {:onyx/name :identity 20 | :onyx/fn :clojure.core/identity 21 | :onyx/type :function} 22 | batch-settings)] 23 | :lifecycles [] 24 | :windows [] 25 | :triggers [] 26 | :flow-conditions [] 27 | :task-scheduler :onyx.task-scheduler/balanced})] 28 | (-> base-job 29 | (add-task (consumer :read-messages 30 | (merge {:kafka/topic topic 31 | :kafka/group-id "onyx-consumer" 32 | :kafka/zookeeper zk-address 33 | :kafka/offset-reset :earliest 34 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 35 | :onyx/min-peers 1 36 | :onyx/max-peers 1} 37 | batch-settings))) 38 | (add-task (core-async/output :out batch-settings))))) 39 | 40 | (deftest kafka-static-partition-test 41 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 42 | _ (println "Using topic" test-topic) 43 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 44 | zk-address (get-in peer-config [:zookeeper/address]) 45 | tenancy-id (str (java.util.UUID/randomUUID)) 46 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 47 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 48 | job (build-job zk-address test-topic 10 1000) 49 | {:keys [out read-messages]} (get-core-async-channels job) 50 | test-data [{:n 1} {:n 2} {:n 3} :done]] 51 | (with-test-env [test-env [4 env-config peer-config]] 52 | (onyx.test-helper/validate-enough-peers! test-env job) 53 | (test-utils/write-data test-topic zk-address (:kafka-bootstrap test-config) test-data) 54 | (->> (onyx.api/submit-job peer-config job) 55 | :job-id 56 | (onyx.test-helper/feedback-exception! peer-config)) 57 | (is (= (butlast test-data) 58 | (onyx.plugin.core-async/take-segments! out 50)))))) 59 | -------------------------------------------------------------------------------- /test/onyx/plugin/input_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.input-test 2 | (:require [clojure.test :refer [deftest is]] 3 | [com.stuartsierra.component :as component] 4 | [onyx.kafka.helpers :as h] 5 | [onyx.test-helper :refer [with-test-env]] 6 | [onyx.job :refer [add-task]] 7 | [onyx.tasks.kafka :refer [consumer]] 8 | [onyx.tasks.core-async :as core-async] 9 | [onyx.plugin.core-async :refer [get-core-async-channels]] 10 | [onyx.plugin.test-utils :as test-utils] 11 | [onyx.plugin.kafka] 12 | [onyx.api])) 13 | 14 | (def n-partitions 4) 15 | 16 | (defn build-job [zk-address topic batch-size batch-timeout] 17 | (let [batch-settings {:onyx/batch-size batch-size :onyx/batch-timeout batch-timeout} 18 | base-job (merge {:workflow [[:read-messages :identity] 19 | [:identity :out]] 20 | :catalog [(merge {:onyx/name :identity 21 | :onyx/fn :clojure.core/identity 22 | :onyx/type :function} 23 | batch-settings)] 24 | :lifecycles [] 25 | :windows [] 26 | :triggers [] 27 | :flow-conditions [] 28 | :task-scheduler :onyx.task-scheduler/balanced})] 29 | (-> base-job 30 | (add-task (consumer :read-messages 31 | (merge {:kafka/topic topic 32 | :kafka/group-id "onyx-consumer" 33 | :kafka/zookeeper zk-address 34 | :kafka/offset-reset :earliest 35 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 36 | :onyx/min-peers n-partitions 37 | :onyx/max-peers n-partitions} 38 | batch-settings))) 39 | (add-task (core-async/output :out batch-settings))))) 40 | 41 | (defn write-data 42 | [topic zookeeper bootstrap-servers] 43 | (h/create-topic! zookeeper topic n-partitions 1) 44 | (let [producer-config {"bootstrap.servers" bootstrap-servers} 45 | key-serializer (h/byte-array-serializer) 46 | value-serializer (h/byte-array-serializer)] 47 | (with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)] 48 | (with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)] 49 | (doseq [x (range 3)] ;0 1 2 50 | (h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x})))) 51 | (doseq [x (range 3)] ;3 4 5 52 | (h/send-sync! producer2 topic nil nil (.getBytes (pr-str {:n (+ 3 x)})))))))) 53 | 54 | (deftest kafka-input-test 55 | (let [test-topic (str (java.util.UUID/randomUUID)) 56 | _ (println "Using topic" test-topic) 57 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 58 | tenancy-id (str (java.util.UUID/randomUUID)) 59 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 60 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 61 | zk-address (get-in peer-config [:zookeeper/address]) 62 | job (build-job zk-address test-topic 10 1000) 63 | {:keys [out read-messages]} (get-core-async-channels job)] 64 | (with-test-env [test-env [(+ n-partitions 2) env-config peer-config]] 65 | (onyx.test-helper/validate-enough-peers! test-env job) 66 | (write-data test-topic zk-address (:kafka-bootstrap test-config)) 67 | (let [job-id (:job-id (onyx.api/submit-job peer-config job))] 68 | (let [results (onyx.plugin.core-async/take-segments! out 10000)] 69 | (println "Result" results) 70 | (is (= 15 (reduce + (mapv :n results))))) 71 | (println "Done taking segments") 72 | (onyx.api/kill-job peer-config job-id))))) 73 | -------------------------------------------------------------------------------- /test/onyx/plugin/output_bench_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.output-bench-test 2 | (:require [clojure.core.async :refer [!! close!]] 3 | [clojure.test :refer [deftest is testing]] 4 | [com.stuartsierra.component :as component] 5 | [onyx.kafka.helpers :as h] 6 | [onyx.test-helper :refer [with-test-env]] 7 | [onyx.job :refer [add-task]] 8 | [onyx.tasks.kafka :refer [producer]] 9 | [onyx.tasks.core-async :as core-async] 10 | [onyx.plugin.core-async :refer [get-core-async-channels]] 11 | [onyx.plugin.test-utils :as test-utils] 12 | [onyx.plugin.kafka] 13 | [onyx.api] 14 | [aero.core :refer [read-config]] 15 | [taoensso.nippy :as nip] 16 | [taoensso.timbre :as log])) 17 | 18 | (def compress-opts {:v1-compatibility? false :compressor nil :encryptor nil :password nil}) 19 | 20 | (defn compress [x] 21 | (nip/freeze x compress-opts)) 22 | 23 | (def decompress-opts {:v1-compatibility? false :compressor nil :encryptor nil :password nil}) 24 | 25 | (defn decompress [x] 26 | (nip/thaw x decompress-opts)) 27 | 28 | (def n-messages-total 1000000) 29 | 30 | (defn wrap-message [segment] 31 | {:message segment}) 32 | 33 | (defn build-job [zk-address topic batch-size batch-timeout] 34 | (let [batch-settings {:onyx/batch-size batch-size 35 | :onyx/batch-timeout batch-timeout} 36 | base-job (merge {:workflow [[:in :write-messages]] 37 | :catalog [] 38 | :lifecycles [] 39 | :windows [] 40 | :triggers [] 41 | :flow-conditions [] 42 | :task-scheduler :onyx.task-scheduler/balanced})] 43 | (-> base-job 44 | (add-task (core-async/input :in batch-settings 10000000)) 45 | (add-task (producer :write-messages 46 | (merge {:kafka/topic topic 47 | :kafka/zookeeper zk-address 48 | :kafka/serializer-fn ::compress 49 | :onyx/batch-size 500 50 | :onyx/fn ::wrap-message 51 | :kafka/request-size 307200} 52 | batch-settings)))))) 53 | 54 | (deftest ^:benchmark kafka-output-benchmark-test 55 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 56 | {:keys [env-config peer-config]} (read-config (clojure.java.io/resource "config.edn") 57 | {:profile :bench}) 58 | tenancy-id (java.util.UUID/randomUUID) 59 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 60 | peer-group (onyx.api/start-peer-group peer-config) 61 | v-peers (onyx.api/start-peers 2 peer-group) 62 | zk-address (get-in peer-config [:zookeeper/address]) 63 | job (build-job zk-address test-topic 500 5) 64 | {:keys [in]} (get-core-async-channels job) 65 | mock (atom {}) 66 | test-data (mapv (fn [v] {:n v}) (range n-messages-total))] 67 | (try 68 | (println "Spooling test data") 69 | (run! #(>!! in %) test-data) 70 | (close! in) 71 | (println "test data out") 72 | (let [start-time (System/currentTimeMillis)] 73 | (->> (onyx.api/submit-job peer-config job) 74 | :job-id 75 | (onyx.test-helper/feedback-exception! peer-config)) 76 | (testing "routing to default topic" 77 | (log/info "Waiting on messages in" test-topic) 78 | (let [run-time (- (System/currentTimeMillis) start-time) 79 | _ (println (float (* 1000 (/ n-messages-total run-time))) "messages per second. Processed" n-messages-total "messages in" run-time "ms.") 80 | ;msgs (take-now zk-address test-topic decompress 180000) 81 | ] 82 | ;(is (= (map :n test-data) (map :value msgs))) 83 | ))) 84 | (finally 85 | (log/info "Stopping mock Kafka...") 86 | (doseq [p v-peers] 87 | (onyx.api/shutdown-peer p)) 88 | (onyx.api/shutdown-peer-group peer-group))))) 89 | -------------------------------------------------------------------------------- /test/onyx/plugin/output_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.output-test 2 | (:require [clojure.core.async :refer [!!]] 3 | [clojure.test :refer [deftest is testing]] 4 | [com.stuartsierra.component :as component] 5 | [onyx.test-helper :refer [with-test-env]] 6 | [onyx.job :refer [add-task]] 7 | [onyx.kafka.helpers :as h] 8 | [onyx.tasks.kafka :refer [producer]] 9 | [onyx.tasks.core-async :as core-async] 10 | [onyx.plugin.core-async :refer [get-core-async-channels]] 11 | [onyx.plugin.test-utils :as test-utils] 12 | [onyx.plugin.kafka] 13 | [onyx.api] 14 | [taoensso.timbre :as log])) 15 | 16 | (defn build-job [zk-address topic batch-size batch-timeout] 17 | (let [batch-settings {:onyx/batch-size batch-size 18 | :onyx/batch-timeout batch-timeout} 19 | base-job (merge {:workflow [[:in :identity] 20 | [:identity :write-messages]] 21 | :catalog [(merge {:onyx/name :identity 22 | :onyx/fn :clojure.core/identity 23 | :onyx/type :function} 24 | batch-settings)] 25 | :lifecycles [] 26 | :windows [] 27 | :triggers [] 28 | :flow-conditions [] 29 | :task-scheduler :onyx.task-scheduler/balanced})] 30 | (-> base-job 31 | (add-task (core-async/input :in batch-settings)) 32 | (add-task (producer :write-messages 33 | (merge {:kafka/topic topic 34 | :kafka/zookeeper zk-address 35 | :kafka/serializer-fn :onyx.tasks.kafka/serialize-message-edn 36 | :kafka/key-serializer-fn :onyx.tasks.kafka/serialize-message-edn 37 | :kafka/request-size 307200} 38 | batch-settings)))))) 39 | 40 | (defn- decompress 41 | [v] 42 | (when v 43 | (read-string (String. v "UTF-8")))) 44 | 45 | (defn- prepare-messages 46 | [coll & extra-keys] 47 | (log/infof "Preparing %d messages..." (count coll)) 48 | (->> coll 49 | (sort-by (comp :n :value)) 50 | (map #(select-keys % (into [:key :partition :topic :value] extra-keys))))) 51 | 52 | (deftest kafka-output-test 53 | (let [test-topic (str "onyx-test-" (java.util.UUID/randomUUID)) 54 | other-test-topic (str "onyx-test-other-" (java.util.UUID/randomUUID)) 55 | timestamp-test-topic (str "onyx-test-other-" (java.util.UUID/randomUUID)) 56 | test-timestamp (System/currentTimeMillis) 57 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 58 | tenancy-id (str (java.util.UUID/randomUUID)) 59 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 60 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 61 | zk-address (get-in peer-config [:zookeeper/address]) 62 | job (build-job zk-address test-topic 10 1000) 63 | bootstrap-servers (:kafka-bootstrap test-config) 64 | {:keys [in]} (get-core-async-channels job) 65 | test-data [{:key 1 :message {:n 0}} 66 | {:message {:n 1}} 67 | {:key "tarein" :message {:n 2}} 68 | {:message {:n 3} :topic other-test-topic} 69 | {:message {:n 4} :topic timestamp-test-topic :timestamp test-timestamp}]] 70 | (with-test-env [test-env [4 env-config peer-config]] 71 | (onyx.test-helper/validate-enough-peers! test-env job) 72 | (h/create-topic! zk-address test-topic 1 1) 73 | (h/create-topic! zk-address other-test-topic 1 1) 74 | (h/create-topic! zk-address timestamp-test-topic 1 1) 75 | (run! #(>!! in %) test-data) 76 | (close! in) 77 | (->> (onyx.api/submit-job peer-config job) 78 | :job-id 79 | (onyx.test-helper/feedback-exception! peer-config)) 80 | (testing "routing to default topic" 81 | (log/info "Waiting on messages in" test-topic) 82 | (let [msgs (prepare-messages 83 | (h/take-now bootstrap-servers test-topic decompress 15000))] 84 | (is (= [test-topic] (->> msgs (map :topic) distinct))) 85 | (is (= [{:key 1 :value {:n 0} :partition 0} 86 | {:key nil :value {:n 1} :partition 0} 87 | {:key "tarein" :value {:n 2} :partition 0}] 88 | (map #(dissoc % :topic) msgs))))) 89 | (testing "overriding the topic" 90 | (log/info "Waiting on messages in" other-test-topic) 91 | (is (= [{:key nil :value {:n 3} :partition 0 :topic other-test-topic}] 92 | (prepare-messages 93 | (h/take-now bootstrap-servers other-test-topic decompress))))) 94 | (testing "overriding the timestamp" 95 | (log/info "Waiting on messages in" timestamp-test-topic) 96 | (is (= [{:key nil :value {:n 4} :partition 0 :topic timestamp-test-topic :timestamp test-timestamp}] 97 | (prepare-messages (h/take-now bootstrap-servers timestamp-test-topic decompress) :timestamp))))))) 98 | -------------------------------------------------------------------------------- /test/onyx/plugin/partition_assignment_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.partition-assignment-test 2 | (:require [clojure.test :refer [deftest is testing]] 3 | [clojure.test.check.generators :as gen] 4 | [com.gfredericks.test.chuck :refer [times]] 5 | [com.gfredericks.test.chuck.clojure-test :refer [checking]] 6 | [onyx.plugin.partition-assignment :refer [partitions-for-slot]])) 7 | 8 | (deftest test-slot-assigment 9 | (let [parts 1 10 | slots 1] 11 | (is (= [0 0] (partitions-for-slot parts slots 0)))) 12 | 13 | (let [parts 3 14 | slots 1] 15 | (is (= [0 2] (partitions-for-slot parts slots 0)))) 16 | 17 | (let [parts 4 18 | slots 2] 19 | (is (= [0 1] (partitions-for-slot parts slots 0))) 20 | (is (= [2 3] (partitions-for-slot parts slots 1)))) 21 | 22 | (let [parts 4 23 | slots 3] 24 | (is (= [0 1] (partitions-for-slot parts slots 0))) 25 | (is (= [2 2] (partitions-for-slot parts slots 1))) 26 | (is (= [3 3] (partitions-for-slot parts slots 2)))) 27 | 28 | (let [parts 7 29 | slots 3] 30 | (is (= [0 2] (partitions-for-slot parts slots 0))) 31 | (is (= [3 4] (partitions-for-slot parts slots 1))) 32 | (is (= [5 6] (partitions-for-slot parts slots 2)))) 33 | 34 | (let [parts 11 35 | slots 3] 36 | (is (= [0 3] (partitions-for-slot parts slots 0))) 37 | (is (= [4 7] (partitions-for-slot parts slots 1))) 38 | (is (= [8 10] (partitions-for-slot parts slots 2))))) 39 | 40 | (deftest full-partition-coverage 41 | (checking 42 | "All partitions are assigned" 43 | (times 50000) 44 | [[parts slots] 45 | (gen/bind gen/s-pos-int 46 | (fn [parts] 47 | (gen/bind (gen/resize (dec parts) gen/s-pos-int) 48 | (fn [slots] (gen/return [parts slots])))))] 49 | (is 50 | (= parts 51 | (reduce 52 | (fn [sum slot] 53 | (let [[lower upper] (partitions-for-slot parts slots slot)] 54 | (+ sum (inc (- upper lower))))) 55 | 0 56 | (range slots)))))) 57 | 58 | (deftest contiguous-assignments 59 | (checking 60 | "Partitions are contiguously assigned" 61 | (times 50000) 62 | [[parts slots] 63 | (gen/bind gen/s-pos-int 64 | (fn [parts] 65 | (gen/bind (gen/resize (dec parts) gen/s-pos-int) 66 | (fn [slots] (gen/return [parts slots])))))] 67 | (let [partitions (map (partial partitions-for-slot parts slots) (range slots))] 68 | (doseq [[a b] (partition 2 1 partitions)] 69 | (is (= (second a) (dec (first b)))))))) 70 | -------------------------------------------------------------------------------- /test/onyx/plugin/target_offsets_test.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.target-offsets-test 2 | (:require [clojure.test :refer [deftest is testing]] 3 | [com.stuartsierra.component :as component] 4 | [onyx.kafka.helpers :as h] 5 | [onyx.test-helper :refer [with-test-env]] 6 | [onyx.job :refer [add-task]] 7 | [onyx.tasks.kafka :refer [consumer]] 8 | [onyx.tasks.core-async :as core-async] 9 | [onyx.plugin.core-async :refer [get-core-async-channels]] 10 | [onyx.plugin.test-utils :as test-utils] 11 | [onyx.plugin.kafka] 12 | [onyx.api]) 13 | (:import [org.apache.kafka.common.errors TopicExistsException])) 14 | 15 | (def n-partitions (long 4)) 16 | 17 | (defn build-job [zk-address topic batch-size batch-timeout] 18 | (let [batch-settings {:onyx/batch-size batch-size :onyx/batch-timeout batch-timeout} 19 | base-job (merge {:workflow [[:read-messages :identity] 20 | [:identity :out]] 21 | :catalog [(merge {:onyx/name :identity 22 | :onyx/fn :clojure.core/identity 23 | :onyx/type :function} 24 | batch-settings)] 25 | :lifecycles [] 26 | :windows [] 27 | :triggers [] 28 | :flow-conditions [] 29 | :task-scheduler :onyx.task-scheduler/balanced})] 30 | (-> base-job 31 | (add-task (consumer :read-messages 32 | (merge {:kafka/topic topic 33 | :kafka/group-id "onyx-consumer" 34 | :kafka/zookeeper zk-address 35 | :kafka/offset-reset :earliest 36 | :kafka/deserializer-fn :onyx.tasks.kafka/deserialize-message-edn 37 | :onyx/min-peers n-partitions 38 | :onyx/max-peers n-partitions 39 | :kafka/wrap-with-metadata? true 40 | :kafka/target-offsets {0 20 41 | 1 25 42 | 2 30 43 | 3 35}} 44 | batch-settings))) 45 | (add-task (core-async/output :out batch-settings))))) 46 | 47 | (defn write-data 48 | [topic zookeeper bootstrap-servers] 49 | (try (h/create-topic! zookeeper topic n-partitions 1) 50 | (catch TopicExistsException _ 51 | (println "Topic exists") 52 | nil)) 53 | (let [producer-config {"bootstrap.servers" bootstrap-servers} 54 | key-serializer (h/byte-array-serializer) 55 | value-serializer (h/byte-array-serializer)] 56 | (with-open [producer1 (h/build-producer producer-config key-serializer value-serializer)] 57 | (with-open [producer2 (h/build-producer producer-config key-serializer value-serializer)] 58 | (doseq [x (range 200)] ;0 1 2 59 | (h/send-sync! producer1 topic nil nil (.getBytes (pr-str {:n x})))) 60 | (doseq [x (range 180)] ;3 4 5 61 | (h/send-sync! producer2 topic nil nil (.getBytes (pr-str {:n (+ 3 (long x))})))))))) 62 | 63 | (deftest kafka-target-offsets-test 64 | (let [test-topic (str (java.util.UUID/randomUUID)) 65 | _ (println "Using topic" test-topic) 66 | {:keys [test-config env-config peer-config]} (onyx.plugin.test-utils/read-config) 67 | tenancy-id (str (java.util.UUID/randomUUID)) 68 | env-config (assoc env-config :onyx/tenancy-id tenancy-id) 69 | peer-config (assoc peer-config :onyx/tenancy-id tenancy-id) 70 | zk-address (get-in peer-config [:zookeeper/address]) 71 | job (build-job zk-address test-topic 10 1000) 72 | {:keys [out read-messages]} (get-core-async-channels job)] 73 | (with-test-env [test-env [(+ (long n-partitions) 2) env-config peer-config]] 74 | (onyx.test-helper/validate-enough-peers! test-env job) 75 | (write-data test-topic zk-address (:kafka-bootstrap test-config)) 76 | (let [job-id (:job-id (onyx.api/submit-job peer-config job))] 77 | (let [first-results (onyx.plugin.core-async/take-segments! out 10000) 78 | partition-count-map (into {} 79 | (map (fn [[k v]] 80 | [k (count v)])) 81 | (group-by :partition first-results))] 82 | (testing "We get the exact ammount of records requested, plus the final markers." 83 | (is (= (get partition-count-map 0) 22)) 84 | (is (= (get partition-count-map 1) 27)) 85 | (is (= (get partition-count-map 2) 32)) 86 | (is (= (get partition-count-map 3) 37)))) 87 | (println "Done taking segments") 88 | (onyx.api/kill-job peer-config job-id))))) 89 | -------------------------------------------------------------------------------- /test/onyx/plugin/test_utils.clj: -------------------------------------------------------------------------------- 1 | (ns onyx.plugin.test-utils 2 | (:require [clojure.core.async :refer [