├── LICENSE ├── NOTICE ├── README.md ├── pom.xml └── src ├── main ├── assemblies │ └── plugin.xml ├── java │ └── org │ │ └── elasticsearch │ │ ├── plugin │ │ └── river │ │ │ └── kafka │ │ │ └── KafkaRiverPlugin.java │ │ └── river │ │ └── kafka │ │ ├── JsonMessageHandler.java │ │ ├── JsonMessageHandlerFactory.java │ │ ├── KafkaClient.java │ │ ├── KafkaRiver.java │ │ ├── KafkaRiverConfig.java │ │ ├── KafkaRiverModule.java │ │ ├── MessageHandler.java │ │ ├── MessageHandlerFactory.java │ │ ├── RawMessageHandler.java │ │ ├── Stats.java │ │ └── StatsReporter.java └── resources │ └── es-plugin.properties └── test ├── java └── org │ └── elasticsearch │ └── river │ └── kafka │ ├── JsonMessageHandlerFactoryTest.java │ ├── JsonMessageHandlerTest.java │ ├── KafkaClientTest.java │ ├── KafkaRiverConfigTest.java │ ├── RawMessageHandlerTest.java │ ├── StatsReporterTest.java │ └── StatsTest.java └── resources └── log4j.properties /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | elasticsearch-river-kafka 2 | 3 | Copyright 2009-2013 Shay Banon and ElasticSearch (http://www.elasticsearch.org) 4 | Copyright 2013 Endgame, Inc. (http://www.endgame.com/) 5 | 6 | This product includes software plugin developed for 7 | ElasticSearch and Shay Banon – (http://http://www.elasticsearch.org/) 8 | 9 | Inspiration was taken from 10 | https://github.com/elasticsearch/elasticsearch-river-rabbitmq 11 | 12 | Licensed under Apache License, Version 2.0 13 | 14 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Kafka River Plugin for ElasticSearch 2 | ================================== 3 | 4 | The Kafka River plugin allows index bulk format messages into elasticsearch. 5 | 6 | 1. Download & Build Kafka 7 | 8 | See [Apacke Kafka Quick Start Guide](http://kafka.apache.org/07/quickstart.html) for instructions on how to Download and Build. 9 | 10 | If you are installing on an encyrpted Ubuntu drive you may get "File name too long" error during the build. 11 | This can be solved by building on an unencrypted file system and moving the files to your desired install point. 12 | 13 | 2. install kafka in your maven repo: 14 | 15 | mvn install:install-file -Dfile=./core/target/scala_2.8.0/kafka-0.7.2.jar -DgroupId=org.apache.kafka \ 16 | -DartifactId=kafka -Dversion=0.7.2 -Dpackaging=jar 17 | 18 | 3. Build this plugin: 19 | 20 | mvn compile test package 21 | # this will create a file here: target/releases/elasticsearch-river-kafka-1.0.1-SNAPSHOT.zip 22 | PLUGIN_PATH=`pwd`/target/releases/elasticsearch-river-kafka-1.0.1-SNAPSHOT.zip 23 | 24 | 4. Install the PLUGIN 25 | 26 | cd $ELASTICSEARCH_HOME 27 | ./bin/plugin -url file:/$PLUGIN_PATH -install elasticsearch-river-kafka 28 | 29 | 5. Updating the plugin 30 | 31 | cd $ELASTICSEARCH_HOME 32 | ./bin/plugin -remove elasticsearch-river-kafka 33 | ./bin/plugin -url file:/$PLUGIN_PATH -install elasticsearch-river-kafka 34 | 35 | ##### Version Support 36 | 37 | ElasticSearch version 0.90.0 38 | 39 | 40 | Deployment 41 | ========== 42 | 43 | Creating the Kafka river is as simple as (all configuration parameters are provided, with default values): 44 | 45 | bulk_size_bytes - max size of messages to pull from Kafka each request 46 | bulk_timeout - socket timeout for Kafka 47 | 48 | curl -XPUT 'localhost:9200/_river/my_kafka_river_0/_meta' -d '{ 49 | "type" : "kafka", 50 | "kafka" : { 51 | "broker_host" : "localhost", 52 | "message_handler_factory_class" : "org.elasticsearch.river.kafka.JsonMessageHandlerFactory", 53 | "zookeeper" : "localhost", 54 | "topic" : "my_topic", 55 | "partition" : "0", 56 | "broker_port" : 9092 57 | }, 58 | "index" : { 59 | "bulk_size_bytes" : 10000000, 60 | "bulk_timeout" : "1000ms" 61 | }, 62 | "statsd":{ 63 | "prefix": "es-kafka-river", 64 | "host": "ambassador", 65 | "port": "8125" 66 | } 67 | 68 | }' 69 | 70 | Kafka offsets are stored in zookeeper. 71 | 72 | NOTE: in its current form, this River only reads from a single broker and a single partition. This will likely change in the future. In 73 | order to consume from multiple partitions and multiple brokers, multiple rivers need to be configured. 74 | 75 | curl -XPUT 'localhost:9200/_river/my_kafka_river_0/_meta' -d '{ 76 | "type" : "kafka", 77 | "kafka" : { 78 | "broker_host" : "localhost", 79 | "message_handler_factory_class" : "org.elasticsearch.river.kafka.JsonMessageHandlerFactory", 80 | "zookeeper" : "localhost", 81 | "topic" : "my_topic", 82 | "partition" : "0", 83 | "broker_port" : 9092 84 | }, 85 | "index" : { 86 | "bulk_size_bytes" : 10000000, 87 | "bulk_timeout" : "1000ms" 88 | }, 89 | "statsd":{ 90 | "prefix": "es-kafka-river", 91 | "host": "ambassador", 92 | "port": "8125" 93 | } 94 | 95 | }' 96 | curl -XPUT 'localhost:9200/_river/my_kafka_river_1/_meta' -d '{ 97 | "type" : "kafka", 98 | "kafka" : { 99 | "broker_host" : "localhost", 100 | "message_handler_factory_class" : "org.elasticsearch.river.kafka.JsonMessageHandlerFactory", 101 | "zookeeper" : "localhost", 102 | "topic" : "my_topic", 103 | "partition" : "1", 104 | "broker_port" : 9092 105 | }, 106 | "index" : { 107 | "bulk_size_bytes" : 10000000, 108 | "bulk_timeout" : "1000ms" 109 | }, 110 | "statsd":{ 111 | "prefix": "es-kafka-river", 112 | "host": "ambassador", 113 | "port": "8125" 114 | } 115 | 116 | }' 117 | curl -XPUT 'localhost:9200/_river/my_kafka_river_2/_meta' -d '{ 118 | "type" : "kafka", 119 | "kafka" : { 120 | "broker_host" : "localhost", 121 | "message_handler_factory_class" : "org.elasticsearch.river.kafka.JsonMessageHandlerFactory", 122 | "zookeeper" : "localhost", 123 | "topic" : "my_topic", 124 | "partition" : "2", 125 | "broker_port" : 9092 126 | }, 127 | "index" : { 128 | "bulk_size_bytes" : 10000000, 129 | "bulk_timeout" : "1000ms" 130 | }, 131 | "statsd":{ 132 | "prefix": "es-kafka-river", 133 | "host": "ambassador", 134 | "port": "8125" 135 | } 136 | 137 | }' 138 | curl -XPUT 'localhost:9200/_river/my_kafka_river_3/_meta' -d '{ 139 | "type" : "kafka", 140 | "kafka" : { 141 | "broker_host" : "localhost", 142 | "message_handler_factory_class" : "org.elasticsearch.river.kafka.JsonMessageHandlerFactory", 143 | "zookeeper" : "localhost", 144 | "topic" : "my_topic", 145 | "partition" : "3", 146 | "broker_port" : 9092 147 | }, 148 | "index" : { 149 | "bulk_size_bytes" : 10000000, 150 | "bulk_timeout" : "1000ms" 151 | }, 152 | "statsd":{ 153 | "prefix": "es-kafka-river", 154 | "host": "ambassador", 155 | "port": "8125" 156 | } 157 | 158 | }' 159 | 160 | The river is automatically bulking queue messages if the queue is overloaded, allowing for faster catchup with the 161 | messages streamed into the queue. The `ordered` flag allows to make sure that the messages will be indexed in the 162 | same order as they arrive in the query by blocking on the bulk request before picking up the next data to be indexed. 163 | It can also be used as a simple way to throttle indexing. 164 | 165 | If `message_handler_factory_class` is not set it will use the `JsonMessageHandlerFactory` and will expect json messages from Kafka with this format: 166 | 167 | { 168 | "index" : "example_index", 169 | "type" : "example_type", 170 | "id" : "asdkljflkasjdfasdfasdf", 171 | "source" : { ..... } 172 | } 173 | 174 | License 175 | ------- 176 | 177 | elasticsearch-river-kafka 178 | 179 | Copyright 2013 [Endgame, Inc.](http://www.endgame.com/) 180 | 181 | ![Endgame, Inc.](http://www.endgame.com/images/navlogo.png) 182 | 183 | This product includes software plugin developed for 184 | ElasticSearch and Shay Banon – [Elasticsearch](http://www.elasticsearch.org/) 185 | 186 | Inspiration was taken from David Pilato and his ElasticSearch Rabbit MQ Plugin 187 | https://github.com/elasticsearch/elasticsearch-river-rabbitmq 188 | 189 | Licensed under the Apache License, Version 2.0 (the "License"); you may 190 | not use this file except in compliance with the License. You may obtain 191 | a copy of the License at 192 | 193 | http://www.apache.org/licenses/LICENSE-2.0 194 | 195 | Unless required by applicable law or agreed to in writing, 196 | software distributed under the License is distributed on an 197 | "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 198 | KIND, either express or implied. See the License for the 199 | specific language governing permissions and limitations 200 | under the License. 201 | 202 | Contributors 203 | ------------- 204 | 205 | - [Jason Trost](https://github.com/jt6211/) 206 | - [Mark Conlin](https://github.com/meconlin) 207 | 208 | 209 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | elasticsearch-river-kafka 4 | 4.0.0 5 | org.elasticsearch 6 | elasticsearch-river-kafka 7 | 1.0.2-SNAPSHOT 8 | jar 9 | Kafka River for ElasticSearch 10 | 2013 11 | 12 | 13 | The Apache Software License, Version 2.0 14 | http://www.apache.org/licenses/LICENSE-2.0.txt 15 | repo 16 | 17 | 18 | 19 | 0.90.0 20 | 21 | 22 | 23 | junit 24 | junit 25 | 3.8.1 26 | test 27 | 28 | 29 | org.elasticsearch 30 | elasticsearch 31 | ${elasticsearch.version} 32 | compile 33 | 34 | 35 | org.apache.kafka 36 | kafka 37 | 0.7.2 38 | 39 | 40 | com.netflix.curator 41 | curator-framework 42 | 1.0.1 43 | 44 | 45 | com.timgroup 46 | java-statsd-client 47 | 1.0.1 48 | 49 | 50 | org.codehaus.jackson 51 | jackson-mapper-asl 52 | 1.9.3 53 | 54 | 55 | org.scala-lang 56 | scala-library 57 | 2.8.0 58 | 59 | 60 | net.sf.jopt-simple 61 | jopt-simple 62 | 3.2 63 | 64 | 65 | com.github.sgroschupf 66 | zkclient 67 | 0.1 68 | 69 | 70 | log4j 71 | log4j 72 | 1.2.16 73 | runtime 74 | 75 | 76 | javax.mail 77 | mail 78 | 79 | 80 | javax.jms 81 | jms 82 | 83 | 84 | com.sun.jdmk 85 | jmxtools 86 | 87 | 88 | com.sun.jmx 89 | jmxri 90 | 91 | 92 | 93 | 94 | org.xerial.snappy 95 | snappy-java 96 | 1.0.4.1 97 | 98 | 99 | org.apache.zookeeper 100 | zookeeper 101 | 3.3.4 102 | 103 | 104 | log4j 105 | log4j 106 | 107 | 108 | jline 109 | jline 110 | 111 | 112 | 113 | 114 | org.easymock 115 | easymock 116 | 3.0 117 | test 118 | 119 | 120 | 121 | 122 | 123 | org.apache.maven.plugins 124 | maven-compiler-plugin 125 | 2.3.2 126 | 127 | 1.7 128 | 1.7 129 | 130 | 131 | 132 | org.apache.maven.plugins 133 | maven-source-plugin 134 | 2.1.2 135 | 136 | 137 | attach-sources 138 | 139 | jar 140 | 141 | 142 | 143 | 144 | 145 | maven-assembly-plugin 146 | 2.3 147 | 148 | false 149 | ${project.build.directory}/releases/ 150 | 151 | ${basedir}/src/main/assemblies/plugin.xml 152 | 153 | 154 | 155 | 156 | package 157 | 158 | single 159 | 160 | 161 | 162 | 163 | 164 | org.apache.maven.plugins 165 | maven-surefire-plugin 166 | 2.12 167 | 168 | -XX:-UseSplitVerifier 169 | 170 | 171 | 172 | org.codehaus.mojo 173 | cobertura-maven-plugin 174 | 2.5.1 175 | 176 | 177 | xml 178 | 179 | 180 | 181 | 182 | cobertura 183 | 184 | cobertura 185 | 186 | 187 | 188 | 189 | 190 | 191 | 192 | -------------------------------------------------------------------------------- /src/main/assemblies/plugin.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | plugin 4 | 5 | zip 6 | 7 | false 8 | 9 | 10 | / 11 | true 12 | true 13 | 14 | org.elasticsearch:elasticsearch 15 | 16 | 17 | 18 | / 19 | true 20 | true 21 | 22 | org.apache.kafka:kafka 23 | com.netflix.curator:curator-framework 24 | org.scala-lang:scala-library 25 | org.apache.zookeeper:zookeeper 26 | net.sf.jopt-simple:jopt-simple 27 | com.github.sgroschupf:zkclient 28 | org.xerial.snappy:snappy-java 29 | org.codehaus.jackson:jackson-mapper-asl 30 | log4j:log4j 31 | com.timgroup:java-statsd-client 32 | 33 | 34 | 35 | 36 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/plugin/river/kafka/KafkaRiverPlugin.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.plugin.river.kafka; 17 | 18 | import org.elasticsearch.common.inject.Inject; 19 | import org.elasticsearch.plugins.AbstractPlugin; 20 | import org.elasticsearch.river.RiversModule; 21 | import org.elasticsearch.river.kafka.KafkaRiverModule; 22 | 23 | /** 24 | * 25 | */ 26 | public class KafkaRiverPlugin extends AbstractPlugin { 27 | 28 | @Inject 29 | public KafkaRiverPlugin() { 30 | } 31 | 32 | @Override 33 | public String name() { 34 | return "river-kafka"; 35 | } 36 | 37 | @Override 38 | public String description() { 39 | return "River Kafka Plugin"; 40 | } 41 | 42 | public void onModule(RiversModule module) { 43 | module.registerRiver("kafka", KafkaRiverModule.class); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/JsonMessageHandler.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import java.util.Map; 19 | 20 | import kafka.message.Message; 21 | 22 | import org.codehaus.jackson.map.ObjectMapper; 23 | import org.codehaus.jackson.map.ObjectReader; 24 | import org.codehaus.jackson.type.TypeReference; 25 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 26 | import org.elasticsearch.action.index.IndexRequestBuilder; 27 | import org.elasticsearch.client.Client; 28 | 29 | /** 30 | * JsonMessageHandler 31 | * 32 | * Handle a simple json message 33 | * Uses BulkRequestBuilder to send messages in bulk 34 | * 35 | * example format 36 | * { "index" : "example_index", 37 | * "type" : "example_type", 38 | * "id" : "asdkljflkasjdfasdfasdf", 39 | * "source" : {"source_data1":"values of source_data1", "source_data2" : 99999 } 40 | * } 41 | * 42 | * index, type, and source are required 43 | * id is optional 44 | * 45 | */ 46 | public class JsonMessageHandler extends MessageHandler { 47 | 48 | final ObjectReader reader = new ObjectMapper().reader(new TypeReference>() {}); 49 | 50 | private Client client; 51 | private Map messageMap; 52 | 53 | public JsonMessageHandler(Client client) { 54 | this.client = client; 55 | } 56 | 57 | protected void readMessage(Message message) throws Exception { 58 | messageMap = reader.readValue(getMessageData(message)); 59 | } 60 | 61 | protected String getIndex() { 62 | return (String) messageMap.get("index"); 63 | } 64 | 65 | protected String getType() { 66 | return (String) messageMap.get("type"); 67 | } 68 | 69 | protected String getId() { 70 | return (String) messageMap.get("id"); 71 | } 72 | 73 | protected Map getSource() { 74 | return (Map) messageMap.get("source"); 75 | } 76 | 77 | protected IndexRequestBuilder createIndexRequestBuilder() { 78 | // Note: prepareIndex() will automatically create the index if it 79 | // doesn't exist 80 | return client.prepareIndex(getIndex(), getType(), getId()).setSource(getSource()); 81 | } 82 | 83 | @Override 84 | public void handle(BulkRequestBuilder bulkRequestBuilder, Message message) throws Exception { 85 | this.readMessage(message); 86 | bulkRequestBuilder.add( this.createIndexRequestBuilder() ); 87 | } 88 | 89 | } 90 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/JsonMessageHandlerFactory.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import org.elasticsearch.client.Client; 20 | 21 | public class JsonMessageHandlerFactory implements MessageHandlerFactory { 22 | public MessageHandler createMessageHandler(Client client) throws Exception { 23 | return new JsonMessageHandler(client); 24 | } 25 | } -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/KafkaClient.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import kafka.api.FetchRequest; 20 | import kafka.api.OffsetRequest; 21 | import kafka.javaapi.consumer.SimpleConsumer; 22 | import kafka.javaapi.message.ByteBufferMessageSet; 23 | 24 | import org.apache.zookeeper.CreateMode; 25 | 26 | import com.netflix.curator.framework.CuratorFramework; 27 | import com.netflix.curator.framework.CuratorFrameworkFactory; 28 | import com.netflix.curator.retry.RetryNTimes; 29 | 30 | public class KafkaClient { 31 | 32 | CuratorFramework curator; 33 | SimpleConsumer consumer; 34 | String brokerURL; 35 | 36 | public KafkaClient(String zk, String broker, int port) 37 | { 38 | brokerURL = broker+":"+port; 39 | connect(zk, broker, port); 40 | } 41 | 42 | void connect(String zk, String broker, int port) 43 | { 44 | try { 45 | consumer = new SimpleConsumer(broker, port, 1000, 1024*1024*10); 46 | curator = CuratorFrameworkFactory.newClient(zk, 1000, 15000, new RetryNTimes(5, 2000)); 47 | curator.start(); 48 | } catch (Exception e) { 49 | throw new RuntimeException(e); 50 | } 51 | } 52 | 53 | public void save(String path, String data) 54 | { 55 | try { 56 | if(curator.checkExists().forPath(path) == null){ 57 | curator.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(path, data.getBytes()); 58 | } 59 | else{ 60 | curator.setData().forPath(path, data.getBytes()); 61 | } 62 | } catch (Exception e) { 63 | throw new RuntimeException(e); 64 | } 65 | } 66 | 67 | public String get(String path) { 68 | try { 69 | if (curator.checkExists().forPath(path) != null) { 70 | return new String(curator.getData().forPath(path)); 71 | } else { 72 | return null; 73 | } 74 | } catch (Exception e) { 75 | throw new RuntimeException(e); 76 | } 77 | } 78 | 79 | public void saveOffset(String topic, int partition, long offset) 80 | { 81 | save(String.format("/es-river-kafka/offsets/%s/%s/%d", brokerURL, topic, partition), Long.toString(offset)); 82 | } 83 | 84 | public long getOffset(String topic, int partition) { 85 | String data = get(String.format("/es-river-kafka/offsets/%s/%s/%d", brokerURL, topic, partition)); 86 | if(data == null) 87 | return 0; 88 | return Long.parseLong(data); 89 | } 90 | 91 | public long getNewestOffset(String topic, int partition) { 92 | return consumer.getOffsetsBefore(topic, partition, OffsetRequest.LatestTime(), 1)[0]; 93 | } 94 | 95 | public long getOldestOffset(String topic, int partition) { 96 | return consumer.getOffsetsBefore(topic, partition, OffsetRequest.EarliestTime(), 1)[0]; 97 | } 98 | 99 | ByteBufferMessageSet fetch(String topic, int partition, long offset, int maxSizeBytes) 100 | { 101 | return consumer.fetch(new FetchRequest(topic, partition, offset, maxSizeBytes)); 102 | } 103 | 104 | public void close() { 105 | curator.close(); 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/KafkaRiver.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import kafka.common.InvalidMessageSizeException; 20 | import kafka.common.OffsetOutOfRangeException; 21 | import kafka.javaapi.message.ByteBufferMessageSet; 22 | import kafka.message.MessageAndOffset; 23 | 24 | import org.elasticsearch.action.bulk.BulkItemResponse; 25 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 26 | import org.elasticsearch.action.bulk.BulkResponse; 27 | import org.elasticsearch.client.Client; 28 | import org.elasticsearch.common.inject.Inject; 29 | import org.elasticsearch.common.util.concurrent.EsExecutors; 30 | import org.elasticsearch.river.AbstractRiverComponent; 31 | import org.elasticsearch.river.River; 32 | import org.elasticsearch.river.RiverName; 33 | import org.elasticsearch.river.RiverSettings; 34 | 35 | /** 36 | * KafkaRiver 37 | * 38 | */ 39 | public class KafkaRiver extends AbstractRiverComponent implements River { 40 | 41 | private final Client client; 42 | private final KafkaRiverConfig riverConfig; 43 | 44 | private volatile boolean closed = false; 45 | private volatile Thread thread; 46 | 47 | @Inject 48 | public KafkaRiver(RiverName riverName, RiverSettings settings, Client client) { 49 | super(riverName, settings); 50 | this.client = client; 51 | 52 | try { 53 | logger.info("KafkaRiver created: name={}, type={}", riverName.getName(), riverName.getType()); 54 | this.riverConfig = new KafkaRiverConfig(settings); 55 | } catch (Exception e) { 56 | logger.error("Unexpected Error occurred", e); 57 | throw new RuntimeException(e); 58 | } 59 | } 60 | 61 | @Override 62 | public void start() { 63 | try { 64 | logger.info("creating kafka river: zookeeper = {}, broker = {}, broker_port = {}, message_handler_factory_class = {}", riverConfig.zookeeper, riverConfig.brokerHost, riverConfig.brokerPort, riverConfig.factoryClass); 65 | logger.info("part = {}, topic = {}", riverConfig.partition, riverConfig.topic); 66 | logger.info("bulkSize = {}, bulkTimeout = {}", riverConfig.bulkSize, riverConfig.bulkTimeout); 67 | 68 | KafkaRiverWorker worker = new KafkaRiverWorker(this.createMessageHandler(client, riverConfig), riverConfig, client); 69 | 70 | thread = EsExecutors.daemonThreadFactory(settings.globalSettings(), "kafka_river").newThread(worker); 71 | thread.start(); 72 | } catch (Exception e) { 73 | logger.error("Unexpected Error occurred", e); 74 | throw new RuntimeException(e); 75 | } 76 | } 77 | 78 | @Override 79 | public void close() { 80 | try { 81 | if (closed) { 82 | return; 83 | } 84 | logger.info("closing kafka river"); 85 | closed = true; 86 | thread.interrupt(); 87 | } catch (Exception e) { 88 | logger.error("Unexpected Error occurred", e); 89 | throw new RuntimeException(e); 90 | } 91 | } 92 | 93 | /** 94 | * createMessageHandler 95 | * 96 | * 97 | * @param client 98 | * @param config 99 | * @return 100 | * @throws Exception 101 | */ 102 | private MessageHandler createMessageHandler(Client client, KafkaRiverConfig config) throws Exception{ 103 | MessageHandlerFactory handlerfactory = null; 104 | try { 105 | handlerfactory = (MessageHandlerFactory) Class.forName(config.factoryClass).newInstance(); 106 | } catch (Exception e) { 107 | logger.error("Unexpected Error occurred", e); 108 | throw new RuntimeException(e); 109 | } 110 | 111 | return (MessageHandler) handlerfactory.createMessageHandler(client); 112 | } 113 | 114 | /** 115 | * KafkaRiverWorker 116 | * 117 | * 118 | */ 119 | private class KafkaRiverWorker implements Runnable { 120 | 121 | long offset; 122 | MessageHandler msgHandler; 123 | 124 | private KafkaClient kafka; 125 | private Client client; 126 | private KafkaRiverConfig riverConfig; 127 | 128 | StatsReporter statsd; 129 | private long statsLastPrintTime; 130 | private Stats stats = new Stats(); 131 | 132 | public KafkaRiverWorker(MessageHandler msgHandler, KafkaRiverConfig riverConfig, Client client) 133 | { 134 | this.msgHandler = msgHandler; 135 | this.client = client; 136 | this.riverConfig = riverConfig; 137 | initKakfa(); 138 | resetStats(); 139 | initStatsd(riverConfig); 140 | } 141 | 142 | void initStatsd(KafkaRiverConfig riverConfig) 143 | { 144 | statsd = new StatsReporter(riverConfig); 145 | if(statsd.isEnabled()) 146 | { 147 | logger.info("Created statsd client for prefix={}, host={}, port={}", riverConfig.statsdPrefix, riverConfig.statsdHost, riverConfig.statsdPort); 148 | } 149 | else 150 | { 151 | logger.info("Note: statsd is not configured, only console metrics will be provided"); 152 | } 153 | } 154 | 155 | void resetStats() 156 | { 157 | statsLastPrintTime = System.currentTimeMillis(); 158 | stats.reset(); 159 | } 160 | 161 | void initKakfa() 162 | { 163 | this.kafka = new KafkaClient(riverConfig.zookeeper, riverConfig.brokerHost, riverConfig.brokerPort); 164 | this.offset = kafka.getOffset(riverConfig.topic, riverConfig.partition); 165 | } 166 | 167 | void handleMessages(BulkRequestBuilder bulkRequestBuilder, ByteBufferMessageSet msgs) 168 | { 169 | long numMsg = 0; 170 | for(MessageAndOffset mo : msgs) 171 | { 172 | ++numMsg; 173 | ++stats.numMessages; 174 | try { 175 | msgHandler.handle(bulkRequestBuilder, mo.message()); 176 | } catch (Exception e) { 177 | logger.warn("Failed handling message", e); 178 | } 179 | } 180 | logger.debug("handleMessages processed {} messages", numMsg); 181 | } 182 | 183 | void executeBuilder(BulkRequestBuilder bulkRequestBuilder) 184 | { 185 | if(bulkRequestBuilder.numberOfActions() == 0) 186 | return; 187 | 188 | ++stats.flushes; 189 | BulkResponse response = bulkRequestBuilder.execute().actionGet(); 190 | if (response.hasFailures()) { 191 | logger.warn("failed to execute" + response.buildFailureMessage()); 192 | } 193 | 194 | for(BulkItemResponse resp : response){ 195 | if(resp.isFailed()){ 196 | stats.failed++; 197 | }else{ 198 | stats.succeeded++; 199 | } 200 | } 201 | } 202 | 203 | void processNonEmptyMessageSet(ByteBufferMessageSet msgs) 204 | { 205 | logger.debug("Processing {} bytes of messages ...", msgs.validBytes()); 206 | BulkRequestBuilder bulkRequestBuilder = client.prepareBulk(); 207 | handleMessages(bulkRequestBuilder, msgs); 208 | executeBuilder(bulkRequestBuilder); 209 | offset += msgs.validBytes(); 210 | kafka.saveOffset(riverConfig.topic, riverConfig.partition, offset); 211 | } 212 | 213 | void reconnectToKafka() throws InterruptedException{ 214 | while(true) 215 | { 216 | if(closed) 217 | break; 218 | 219 | try { 220 | try { 221 | kafka.close(); 222 | } catch (Exception e) {} 223 | 224 | initKakfa(); 225 | break; 226 | } 227 | catch(Exception e2){ 228 | logger.error("Error re-connecting to Kafka({}:{}/{}), retrying in 5 sec", e2, riverConfig.brokerHost, riverConfig.topic, riverConfig.partition); 229 | Thread.sleep(5000); 230 | } 231 | } 232 | } 233 | 234 | long getBacklogSize() 235 | { 236 | return kafka.getNewestOffset(riverConfig.topic, riverConfig.partition) - offset; 237 | } 238 | 239 | void dumpStats() 240 | { 241 | long elapsed = System.currentTimeMillis() - statsLastPrintTime; 242 | if(elapsed >= 10000) 243 | { 244 | stats.backlog = getBacklogSize(); 245 | stats.rate = (double)stats.numMessages/((double)elapsed/1000.0); 246 | logger.info("{}:{}/{}:{} {} msg ({} msg/s), flushed {} ({} err, {} succ) [msg backlog {}]", 247 | riverConfig.brokerHost, riverConfig.brokerPort, riverConfig.topic, riverConfig.partition, 248 | stats.numMessages, String.format("%.2f", stats.rate), stats.flushes, 249 | stats.failed, stats.succeeded, 250 | getBytesString(stats.backlog)); 251 | 252 | statsd.reoportStats(stats); 253 | resetStats(); 254 | } 255 | } 256 | 257 | @Override 258 | public void run() { 259 | 260 | try { 261 | logger.info("KafkaRiverWorker is running..."); 262 | 263 | while(true) 264 | { 265 | if(closed) 266 | break; 267 | 268 | dumpStats(); 269 | 270 | try { 271 | ByteBufferMessageSet msgs = kafka.fetch(riverConfig.topic, riverConfig.partition, offset, riverConfig.bulkSize); 272 | if(msgs.validBytes() > 0) 273 | { 274 | processNonEmptyMessageSet(msgs); 275 | } 276 | else 277 | { 278 | logger.debug("No messages received from Kafka for topic={}, partition={}, offset={}, bulkSize={}", 279 | riverConfig.topic, riverConfig.partition, offset, riverConfig.bulkSize); 280 | Thread.sleep(1000); 281 | } 282 | } 283 | catch (InterruptedException e2) { 284 | break; 285 | } 286 | catch(OffsetOutOfRangeException e) 287 | { 288 | // Assumption: EITHER 289 | // 290 | // 1) This River is starting for the first time and Kafka has already aged some data out (so the lowest offset is not 0) 291 | // OR 292 | // 2) This river has gotten far enough behind that Kafka has aged off enough data that the offset is no longer valid. 293 | // If this is the case, this will likely happen everytime Kafka ages off old data unless the data flow decreases in volume. 294 | 295 | logger.warn("Encountered OffsetOutOfRangeException, querying Kafka for oldest Offset and reseting local offset"); 296 | offset = kafka.getOldestOffset(riverConfig.topic, riverConfig.partition); 297 | logger.warn("Setting offset to oldest offset = {}", offset); 298 | } 299 | catch (InvalidMessageSizeException e) { 300 | logger.warn("InvalidMessageSizeException occurred for Kafka({}:{}/{}:{}), querying Kafka for oldest Offset and reseting local offset", e, riverConfig.brokerHost, riverConfig.brokerPort, riverConfig.topic, riverConfig.partition); 301 | offset = kafka.getOldestOffset(riverConfig.topic, riverConfig.partition); 302 | logger.warn("Setting offset to oldest offset = {}", offset); 303 | try { 304 | Thread.sleep(5000); 305 | } catch (InterruptedException e2) { 306 | break; 307 | } 308 | } 309 | catch (Exception e) { 310 | logger.error("Error fetching from Kafka({}:{}/{}:{}), retrying in 5 sec", e, riverConfig.brokerHost, riverConfig.brokerPort, riverConfig.topic, riverConfig.partition); 311 | try { 312 | Thread.sleep(5000); 313 | reconnectToKafka(); 314 | } catch (InterruptedException e2) { 315 | break; 316 | } 317 | } 318 | } // end while 319 | kafka.close(); 320 | logger.info("KafkaRiverWorker is stopping..."); 321 | } catch (Exception e) { 322 | logger.error("Unexpected Error Occurred", e); 323 | 324 | // Don't normally like to rethrow exceptions like this, but ES silently ignores them in Plugins 325 | throw new RuntimeException(e); 326 | } 327 | } // end run 328 | } 329 | 330 | /** 331 | * @param bytes 332 | * @return 333 | */ 334 | static String getBytesString(long bytes) 335 | { 336 | String size; 337 | if( Math.floor(bytes/(1024*1024*1024)) > 0){ 338 | size = String.format("%.2f GB", (double)bytes/(1024.0*1024.0*1024.0)); 339 | } 340 | else if( Math.floor(bytes/(1024*1024)) > 0){ 341 | size = String.format("%.2f MB", (double)bytes/(1024.0*1024.0)); 342 | } 343 | else if( Math.floor(bytes/(1024)) > 0){ 344 | size = String.format("%.2f KB", (double)bytes/(1024.0)); 345 | } 346 | else{ 347 | size = bytes+" B"; 348 | } 349 | return size; 350 | } 351 | } 352 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/KafkaRiverConfig.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import java.util.Map; 20 | 21 | import org.elasticsearch.common.unit.TimeValue; 22 | import org.elasticsearch.common.xcontent.support.XContentMapValues; 23 | import org.elasticsearch.river.RiverSettings; 24 | 25 | public class KafkaRiverConfig { 26 | 27 | public final String zookeeper; 28 | public final String factoryClass; // full class path and name for the concrete message handler class factory 29 | public final String brokerHost; 30 | public final int brokerPort; 31 | public final String topic; 32 | public final int partition; 33 | 34 | public final String statsdPrefix; 35 | public final String statsdHost; 36 | public final int statsdPort; 37 | 38 | public final int bulkSize; 39 | public final TimeValue bulkTimeout; 40 | 41 | public KafkaRiverConfig(RiverSettings settings) 42 | { 43 | if (settings.settings().containsKey("kafka")) { 44 | Map kafkaSettings = (Map) settings.settings().get("kafka"); 45 | 46 | topic = (String)kafkaSettings.get("topic"); 47 | zookeeper = XContentMapValues.nodeStringValue(kafkaSettings.get("zookeeper"), "localhost"); 48 | factoryClass = XContentMapValues.nodeStringValue(kafkaSettings.get("message_handler_factory_class"), "org.elasticsearch.river.kafka.JsonMessageHandlerFactory"); 49 | brokerHost = XContentMapValues.nodeStringValue(kafkaSettings.get("broker_host"), "localhost"); 50 | brokerPort = XContentMapValues.nodeIntegerValue(kafkaSettings.get("broker_port"), 9092); 51 | partition = XContentMapValues.nodeIntegerValue(kafkaSettings.get("partition"), 0); 52 | } 53 | else 54 | { 55 | zookeeper = "localhost"; 56 | brokerHost = "localhost"; 57 | brokerPort = 9092; 58 | topic = "default_topic"; 59 | partition = 0; 60 | factoryClass = "org.elasticsearch.river.kafka.JsonMessageHandlerFactory"; 61 | } 62 | 63 | if (settings.settings().containsKey("index")) { 64 | Map indexSettings = (Map) settings.settings().get("index"); 65 | bulkSize = XContentMapValues.nodeIntegerValue(indexSettings.get("bulk_size_bytes"), 10*1024*1024); 66 | if (indexSettings.containsKey("bulk_timeout")) { 67 | bulkTimeout = TimeValue.parseTimeValue(XContentMapValues.nodeStringValue(indexSettings.get("bulk_timeout"), "10ms"), TimeValue.timeValueMillis(10000)); 68 | } else { 69 | bulkTimeout = TimeValue.timeValueMillis(10); 70 | } 71 | } else { 72 | bulkSize = 10*1024*1024; 73 | bulkTimeout = TimeValue.timeValueMillis(10000); 74 | } 75 | 76 | if (settings.settings().containsKey("statsd")) { 77 | Map statsdSettings = (Map) settings.settings().get("statsd"); 78 | statsdHost = (String)statsdSettings.get("host"); 79 | statsdPort = XContentMapValues.nodeIntegerValue(statsdSettings.get("port"), 8125); 80 | statsdPrefix = XContentMapValues.nodeStringValue(statsdSettings.get("prefix"), "es-kafka-river"); 81 | } 82 | else 83 | { 84 | statsdHost = null; 85 | statsdPort = -1; 86 | statsdPrefix = null; 87 | } 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/KafkaRiverModule.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import org.elasticsearch.common.inject.AbstractModule; 20 | import org.elasticsearch.river.River; 21 | 22 | public class KafkaRiverModule extends AbstractModule { 23 | 24 | @Override 25 | protected void configure() { 26 | bind(River.class).to(KafkaRiver.class).asEagerSingleton(); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/MessageHandler.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import java.nio.ByteBuffer; 20 | 21 | import kafka.message.Message; 22 | 23 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 24 | 25 | public abstract class MessageHandler { 26 | public static byte[] getMessageData(Message message) { 27 | ByteBuffer buf = message.payload(); 28 | byte[] data = new byte[buf.remaining()]; 29 | buf.get(data); 30 | return data; 31 | } 32 | 33 | public abstract void handle(BulkRequestBuilder bulkRequestBuilder, Message message) throws Exception; 34 | 35 | } -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/MessageHandlerFactory.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import org.elasticsearch.client.Client; 20 | 21 | public interface MessageHandlerFactory { 22 | public MessageHandler createMessageHandler(Client client) throws Exception; 23 | } -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/RawMessageHandler.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import kafka.message.Message; 20 | 21 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 22 | 23 | public class RawMessageHandler extends MessageHandler 24 | { 25 | public void handle(BulkRequestBuilder bulkRequestBuilder, Message message) throws Exception 26 | { 27 | byte[] data = getMessageData(message); 28 | bulkRequestBuilder.add(data, 0, data.length, false); 29 | } 30 | } -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/Stats.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | public class Stats { 20 | public int numMessages; 21 | public int flushes; 22 | public int succeeded; 23 | public int failed; 24 | public double rate; 25 | long backlog; 26 | 27 | public Stats(){ 28 | reset(); 29 | } 30 | 31 | void reset() { 32 | numMessages = 0; 33 | flushes = 0; 34 | succeeded = 0; 35 | failed = 0; 36 | rate = 0; 37 | backlog = 0; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/org/elasticsearch/river/kafka/StatsReporter.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package org.elasticsearch.river.kafka; 18 | 19 | import com.timgroup.statsd.StatsDClient; 20 | 21 | public class StatsReporter { 22 | 23 | StatsDClient statsd = null; 24 | 25 | String numMsg; 26 | String flushes; 27 | String failed; 28 | String succeeded; 29 | String rateName; 30 | String backlog; 31 | 32 | public StatsReporter(KafkaRiverConfig riverConfig) { 33 | if(riverConfig.statsdHost == null){ 34 | return; 35 | } 36 | 37 | statsd = new StatsDClient(riverConfig.statsdPrefix, riverConfig.statsdHost, riverConfig.statsdPort); 38 | String baseName = String.format("%s.%d.%s.%d", 39 | riverConfig.brokerHost, riverConfig.brokerPort, 40 | riverConfig.topic, riverConfig.partition); 41 | 42 | numMsg = baseName + ".numMsg"; 43 | flushes = baseName + ".flushes"; 44 | failed = baseName + ".failed"; 45 | succeeded = baseName + ".succeeded"; 46 | rateName = baseName + ".rate"; 47 | backlog = baseName + ".backlog"; 48 | } 49 | 50 | public void reoportStats(Stats stats) { 51 | if(!isEnabled()) 52 | return; 53 | 54 | statsd.count(numMsg, stats.numMessages); 55 | statsd.count(flushes, stats.flushes); 56 | statsd.count(failed, stats.failed); 57 | statsd.count(succeeded, stats.succeeded); 58 | statsd.gauge(rateName, (int) Math.floor(stats.rate)); 59 | if(stats.backlog > Integer.MAX_VALUE){ 60 | statsd.gauge(backlog, Integer.MAX_VALUE); 61 | } 62 | else{ 63 | statsd.gauge(backlog, (int)stats.backlog); 64 | } 65 | } 66 | 67 | boolean isEnabled() { 68 | return null != statsd; 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /src/main/resources/es-plugin.properties: -------------------------------------------------------------------------------- 1 | plugin=org.elasticsearch.plugin.river.kafka.KafkaRiverPlugin 2 | 3 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/JsonMessageHandlerFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import junit.framework.TestCase; 19 | 20 | public class JsonMessageHandlerFactoryTest extends TestCase { 21 | 22 | public void testCreateJsonMessageHandler() throws Exception 23 | { 24 | JsonMessageHandlerFactory jmhf = new JsonMessageHandlerFactory(); 25 | MessageHandler jmh = null; 26 | 27 | try { 28 | jmh = jmhf.createMessageHandler(null); 29 | 30 | } catch (Exception e) { 31 | fail("This should not fail"); 32 | } 33 | 34 | assertEquals(jmh instanceof JsonMessageHandler, true); 35 | } 36 | 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/JsonMessageHandlerTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import static org.easymock.EasyMock.anyObject; 19 | import static org.easymock.EasyMock.createMock; 20 | import static org.easymock.EasyMock.expect; 21 | import static org.easymock.EasyMock.replay; 22 | import static org.easymock.EasyMock.verify; 23 | 24 | import java.nio.ByteBuffer; 25 | import java.util.HashMap; 26 | import java.util.Map; 27 | 28 | import junit.framework.TestCase; 29 | import kafka.message.Message; 30 | 31 | import org.codehaus.jackson.map.ObjectMapper; 32 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 33 | import org.elasticsearch.action.index.IndexRequestBuilder; 34 | import org.elasticsearch.client.Client; 35 | 36 | public class JsonMessageHandlerTest extends TestCase { 37 | 38 | private String toJson(Object value) { 39 | ObjectMapper mapper = new ObjectMapper(); 40 | try { 41 | return mapper.writeValueAsString(value); 42 | } catch (Exception e) { 43 | throw new RuntimeException(e); 44 | } 45 | } 46 | 47 | Map rec = new HashMap(){{ 48 | put("index", "xyz"); 49 | put("type", "datatype1"); 50 | put("source", new HashMap(){{put("field", "1");}}); 51 | }}; 52 | 53 | public void testReadMessage() throws Exception 54 | { 55 | JsonMessageHandler h = new JsonMessageHandler(null); 56 | byte[] json = toJson(rec).getBytes(); 57 | Message message = createMock(Message.class); 58 | expect(message.payload()).andReturn(ByteBuffer.wrap(json)); 59 | replay(message); 60 | 61 | try { 62 | h.readMessage(message); 63 | } catch (Exception e) { 64 | fail("This should not fail"); 65 | } 66 | 67 | verify(message); 68 | } 69 | 70 | public void testGettersFromReadMessageReturnedMap() throws Exception 71 | { 72 | JsonMessageHandler h = new JsonMessageHandler(null); 73 | byte[] json = toJson(rec).getBytes(); 74 | Message message = createMock(Message.class); 75 | 76 | expect(message.payload()).andReturn(ByteBuffer.wrap(json)); 77 | replay(message); 78 | 79 | try { 80 | h.readMessage(message); 81 | } catch (Exception e) { 82 | fail("This should not fail"); 83 | } 84 | 85 | assertEquals(h.getIndex(), rec.get("index")); 86 | assertEquals(h.getType(), rec.get("type")); 87 | assertEquals(h.getSource(), rec.get("source")); 88 | assertEquals(h.getId(), rec.get("id")); 89 | verify(message); 90 | } 91 | 92 | public void testIt() throws Exception 93 | { 94 | Client client = createMock(Client.class); 95 | IndexRequestBuilder irb = createMock(IndexRequestBuilder.class); 96 | JsonMessageHandler h = new JsonMessageHandler(client); 97 | byte[] json = toJson(rec).getBytes(); 98 | 99 | expect(client.prepareIndex(anyObject(String.class), anyObject(String.class), anyObject(String.class))).andReturn(irb); 100 | replay(client); 101 | 102 | Message message = createMock(Message.class); 103 | expect(message.payload()).andReturn(ByteBuffer.wrap(json)); 104 | replay(message); 105 | 106 | BulkRequestBuilder bulkRequestBuilder = createMock(BulkRequestBuilder.class); 107 | 108 | expect(bulkRequestBuilder.add(anyObject(IndexRequestBuilder.class))).andReturn(null); 109 | replay(bulkRequestBuilder); 110 | 111 | try { 112 | h.handle(bulkRequestBuilder, message); 113 | } catch (Exception e) { 114 | fail("This should not fail"); 115 | } 116 | 117 | verify(client); 118 | } 119 | } 120 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/KafkaClientTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import static org.easymock.EasyMock.anyObject; 19 | import static org.easymock.EasyMock.aryEq; 20 | import static org.easymock.EasyMock.createMock; 21 | import static org.easymock.EasyMock.eq; 22 | import static org.easymock.EasyMock.expect; 23 | import static org.easymock.EasyMock.expectLastCall; 24 | import static org.easymock.EasyMock.replay; 25 | import static org.easymock.EasyMock.verify; 26 | 27 | import java.util.Collections; 28 | 29 | import junit.framework.TestCase; 30 | import kafka.api.FetchRequest; 31 | import kafka.javaapi.consumer.SimpleConsumer; 32 | import kafka.javaapi.message.ByteBufferMessageSet; 33 | 34 | import org.apache.zookeeper.data.Stat; 35 | 36 | import com.netflix.curator.framework.CuratorFramework; 37 | import com.netflix.curator.framework.api.ExistsBuilder; 38 | import com.netflix.curator.framework.api.GetDataBuilder; 39 | import com.netflix.curator.framework.api.SetDataBuilder; 40 | 41 | public class KafkaClientTest extends TestCase { 42 | 43 | public ExistsBuilder mockExistsBuilder(String path, boolean exists) throws Exception 44 | { 45 | ExistsBuilder b = createMock(ExistsBuilder.class); 46 | if(exists) 47 | { 48 | expect(b.forPath(eq(path))).andReturn(new Stat()); 49 | } 50 | else 51 | { 52 | expect(b.forPath(eq(path))).andReturn(null); 53 | } 54 | replay(b); 55 | return b; 56 | } 57 | 58 | public GetDataBuilder mockGetDataBuilder(String path, byte[] data) throws Exception 59 | { 60 | GetDataBuilder b = createMock(GetDataBuilder.class); 61 | expect(b.forPath(eq(path))).andReturn(data); 62 | replay(b); 63 | return b; 64 | } 65 | 66 | public SetDataBuilder mockSetDataBuilder(String path, byte[] data) throws Exception 67 | { 68 | SetDataBuilder b = createMock(SetDataBuilder.class); 69 | expect(b.forPath(eq(path), aryEq(data))).andReturn(new Stat()); 70 | replay(b); 71 | return b; 72 | } 73 | 74 | CuratorFramework mockCurator; 75 | SimpleConsumer mockConsumer; 76 | KafkaClient client; 77 | 78 | @Override 79 | protected void setUp() throws Exception { 80 | super.setUp(); 81 | mockCurator = createMock(CuratorFramework.class); 82 | mockConsumer = createMock(SimpleConsumer.class); 83 | 84 | final CuratorFramework cur = mockCurator; 85 | final SimpleConsumer con = mockConsumer; 86 | client = new KafkaClient("zookeeper", "broker", 9092){ 87 | void connect(String zk, String broker, int port) 88 | { 89 | this.curator = cur; 90 | this.consumer = con; 91 | }; 92 | }; 93 | } 94 | 95 | @Override 96 | protected void tearDown() throws Exception { 97 | super.tearDown(); 98 | mockCurator = null; 99 | mockConsumer = null; 100 | client = null; 101 | } 102 | 103 | public void testConstructor() 104 | { 105 | replay(mockConsumer, mockCurator); 106 | assertEquals(client.brokerURL, "broker:9092"); 107 | } 108 | 109 | public void testGetNewestOffset() throws Exception 110 | { 111 | long[] mylong = new long[3]; 112 | mylong[0] = 10; 113 | mylong[1] = 20; 114 | mylong[2] = 30; 115 | 116 | expect(mockConsumer.getOffsetsBefore("topic", 1, -1, 1)).andReturn(mylong); 117 | replay(mockConsumer, mockCurator); 118 | long answer = client.getNewestOffset("topic", 1); 119 | assertEquals(mylong[0], answer); 120 | verify(mockConsumer, mockCurator); 121 | } 122 | 123 | public void testGetOldestOffset() throws Exception 124 | { 125 | long[] mylong = new long[3]; 126 | mylong[0] = 10; 127 | mylong[1] = 20; 128 | mylong[2] = 30; 129 | 130 | expect(mockConsumer.getOffsetsBefore("topic", 1, -2, 1)).andReturn(mylong); 131 | replay(mockConsumer, mockCurator); 132 | long answer = client.getOldestOffset("topic", 1); 133 | assertEquals(mylong[0], answer); 134 | verify(mockConsumer, mockCurator); 135 | } 136 | 137 | public void testGet() throws Exception 138 | { 139 | expect(mockCurator.checkExists()).andReturn(mockExistsBuilder("/some/path", false)); 140 | expect(mockCurator.checkExists()).andReturn(mockExistsBuilder("/some/existing/path", true)); 141 | expect(mockCurator.getData()).andReturn(mockGetDataBuilder("/some/existing/path", "this is data".getBytes())); 142 | 143 | replay(mockConsumer, mockCurator); 144 | assertNull(client.get("/some/path")); 145 | assertEquals(client.get("/some/existing/path"), "this is data"); 146 | verify(mockConsumer, mockCurator); 147 | } 148 | 149 | public void testSave() throws Exception 150 | { 151 | expect(mockCurator.checkExists()).andReturn(mockExistsBuilder("/some/existing/path", true)); 152 | expect(mockCurator.setData()).andReturn(mockSetDataBuilder("/some/existing/path", "this is data".getBytes())); 153 | 154 | replay(mockConsumer, mockCurator); 155 | client.save("/some/existing/path", "this is data"); 156 | verify(mockConsumer, mockCurator); 157 | } 158 | 159 | static class Args 160 | { 161 | String path = ""; 162 | String data = ""; 163 | } 164 | 165 | public void testSaveOffset() 166 | { 167 | final Args args = new Args(); 168 | replay(mockConsumer, mockCurator); 169 | client = new KafkaClient("zookeeper", "broker", 9092){ 170 | void connect(String zk, String broker, int port) 171 | { 172 | this.curator = mockCurator; 173 | this.consumer = mockConsumer; 174 | } 175 | 176 | @Override 177 | public void save(String path, String data) { 178 | args.path = path; 179 | args.data = data; 180 | } 181 | }; 182 | client.saveOffset("my_topic", 77, 4242); 183 | 184 | assertEquals("/es-river-kafka/offsets/broker:9092/my_topic/77", args.path); 185 | assertEquals("4242", args.data); 186 | } 187 | 188 | public void testGetOffsets() 189 | { 190 | final Args args = new Args(); 191 | replay(mockConsumer, mockCurator); 192 | client = new KafkaClient("zookeeper", "broker", 9092){ 193 | void connect(String zk, String broker, int port) 194 | { 195 | this.curator = mockCurator; 196 | this.consumer = mockConsumer; 197 | } 198 | 199 | @Override 200 | public String get(String path) { 201 | args.path = path; 202 | return "100"; 203 | } 204 | }; 205 | 206 | assertEquals(100, client.getOffset("my_topic", 777)); 207 | assertEquals("/es-river-kafka/offsets/broker:9092/my_topic/777", args.path); 208 | } 209 | 210 | public void testClose() 211 | { 212 | mockCurator.close(); 213 | expectLastCall().asStub(); 214 | replay(mockConsumer, mockCurator); 215 | 216 | client.close(); 217 | verify(mockConsumer, mockCurator); 218 | } 219 | 220 | public void testFetch() 221 | { 222 | expect(mockConsumer.fetch(anyObject(FetchRequest.class))).andReturn(new ByteBufferMessageSet(Collections.EMPTY_LIST)); 223 | replay(mockConsumer, mockCurator); 224 | client.fetch("my_topic", 0, 1717, 1024); 225 | verify(mockConsumer, mockCurator); 226 | } 227 | 228 | } 229 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/KafkaRiverConfigTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import java.util.HashMap; 19 | import java.util.Map; 20 | 21 | import org.elasticsearch.common.settings.ImmutableSettings; 22 | import org.elasticsearch.river.RiverSettings; 23 | 24 | import junit.framework.TestCase; 25 | 26 | public class KafkaRiverConfigTest extends TestCase { 27 | public void testIt() 28 | { 29 | Map kafka = new HashMap<>(); 30 | kafka.put("zookeeper", "zoo-host"); 31 | kafka.put("broker_host", "broker-host"); 32 | kafka.put("broker_port", "9999"); 33 | kafka.put("topic", "my_topic"); 34 | kafka.put("partition", "777"); 35 | kafka.put("message_handler_factory_class", "my.factory.class.MyFactory"); 36 | 37 | Map index = new HashMap<>(); 38 | index.put("bulk_size_bytes", "1717171"); 39 | index.put("bulk_timeout", "111ms"); 40 | 41 | Map statsd = new HashMap<>(); 42 | statsd.put("host", "some.host"); 43 | statsd.put("port", "1234"); 44 | statsd.put("prefix", "boo.yeah"); 45 | 46 | Map map = new HashMap<>(); 47 | map.put("kafka", kafka); 48 | map.put("index", index); 49 | map.put("statsd", statsd); 50 | 51 | RiverSettings settings = new RiverSettings( 52 | ImmutableSettings.settingsBuilder().build(), 53 | map); 54 | KafkaRiverConfig c = new KafkaRiverConfig(settings); 55 | 56 | assertEquals("broker-host", c.brokerHost); 57 | assertEquals(9999, c.brokerPort); 58 | assertEquals("zoo-host", c.zookeeper); 59 | assertEquals("my.factory.class.MyFactory", c.factoryClass); 60 | assertEquals(1717171, c.bulkSize); 61 | assertEquals(777, c.partition); 62 | assertEquals("my_topic", c.topic); 63 | assertEquals(111, c.bulkTimeout.millis()); 64 | 65 | 66 | assertEquals(1234, c.statsdPort); 67 | assertEquals("some.host", c.statsdHost); 68 | assertEquals("boo.yeah", c.statsdPrefix); 69 | } 70 | 71 | public void testDefaults() 72 | { 73 | Map map = new HashMap<>(); 74 | RiverSettings settings = new RiverSettings( 75 | ImmutableSettings.settingsBuilder().build(), 76 | map); 77 | KafkaRiverConfig c = new KafkaRiverConfig(settings); 78 | 79 | assertEquals("localhost", c.brokerHost); 80 | assertEquals(9092, c.brokerPort); 81 | assertEquals("localhost", c.zookeeper); 82 | assertEquals(10485760, c.bulkSize); 83 | assertEquals(0, c.partition); 84 | assertEquals("default_topic", c.topic); 85 | assertEquals(10000, c.bulkTimeout.millis()); 86 | 87 | assertEquals("org.elasticsearch.river.kafka.JsonMessageHandlerFactory", c.factoryClass); 88 | 89 | assertEquals(-1, c.statsdPort); 90 | assertNull(c.statsdHost); 91 | assertNull(c.statsdPrefix); 92 | 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/RawMessageHandlerTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import static org.easymock.EasyMock.aryEq; 19 | import static org.easymock.EasyMock.createMock; 20 | import static org.easymock.EasyMock.eq; 21 | import static org.easymock.EasyMock.expect; 22 | import static org.easymock.EasyMock.replay; 23 | import static org.easymock.EasyMock.verify; 24 | 25 | import java.nio.ByteBuffer; 26 | import java.util.ArrayList; 27 | import java.util.List; 28 | 29 | import junit.framework.TestCase; 30 | import kafka.message.Message; 31 | 32 | import org.elasticsearch.action.bulk.BulkRequestBuilder; 33 | 34 | public class RawMessageHandlerTest extends TestCase { 35 | public void testIt() throws Exception 36 | { 37 | byte[] data = "somedata".getBytes(); 38 | List l = new ArrayList<>(); 39 | 40 | MessageHandler m = new RawMessageHandler(); 41 | Message message = createMock(Message.class); 42 | expect(message.payload()).andReturn(ByteBuffer.wrap(data)); 43 | 44 | BulkRequestBuilder bulkRequestBuilder = createMock(BulkRequestBuilder.class); 45 | expect(bulkRequestBuilder.add(aryEq(data), eq(0), eq(data.length), eq(false))).andReturn(null); 46 | replay(message, bulkRequestBuilder); 47 | 48 | m.handle(bulkRequestBuilder, message); 49 | verify(bulkRequestBuilder, message); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/StatsReporterTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import java.util.HashMap; 19 | import java.util.Map; 20 | 21 | import junit.framework.TestCase; 22 | 23 | import org.elasticsearch.common.collect.ImmutableMap; 24 | import org.elasticsearch.common.settings.ImmutableSettings; 25 | import org.elasticsearch.common.settings.Settings; 26 | import org.elasticsearch.river.RiverSettings; 27 | 28 | public class StatsReporterTest extends TestCase { 29 | 30 | public void reoportStats() throws Exception { 31 | Settings globalSettings = ImmutableSettings.settingsBuilder().put("cluster.name", "jason-hfs-cluster").build(); 32 | Map config = new HashMap<>(); 33 | config.put("statsd", ImmutableMap. builder().put("host", "localhost").put("port", "1234").put("prefix", "my_prefix").build()); 34 | 35 | StatsReporter r = new StatsReporter(new KafkaRiverConfig(new RiverSettings(globalSettings, config))); 36 | Stats s = new Stats(); 37 | 38 | // cant mock statsdclient - due to it being final (Easymock creates a 39 | // subclass to work its magic 40 | // 41 | // ALSO count and gauge calls to StatsDClient are no blocking and will 42 | // not throw exception, so basically not much to test here 43 | // 44 | 45 | try { 46 | r.reoportStats(s); 47 | } catch (Exception e) { 48 | fail("This should not fail : " + e.toString()); 49 | } 50 | 51 | s = null; 52 | 53 | boolean thrown = false; 54 | try { 55 | r.reoportStats(s); 56 | fail("Never should have gotten here: "); 57 | } catch (NullPointerException e) { 58 | thrown = true; 59 | } 60 | 61 | assertTrue(thrown); 62 | } 63 | 64 | public void testNormal() { 65 | Settings globalSettings = ImmutableSettings.settingsBuilder().put("cluster.name", "jason-hfs-cluster").build(); 66 | Map config = new HashMap<>(); 67 | config.put("statsd", ImmutableMap. builder().put("host", "localhost").put("port", "1234").put("prefix", "my_prefix").build()); 68 | 69 | StatsReporter r = new StatsReporter(new KafkaRiverConfig(new RiverSettings(globalSettings, config))); 70 | assertTrue(r.isEnabled()); 71 | 72 | } 73 | 74 | public void testNotConfigured() { 75 | Settings globalSettings = ImmutableSettings.settingsBuilder().put("cluster.name", "jason-hfs-cluster").build(); 76 | Map config = new HashMap<>(); 77 | // no statsd config at all 78 | assertFalse(new StatsReporter(new KafkaRiverConfig(new RiverSettings(globalSettings, config))).isEnabled()); 79 | 80 | // missing host 81 | config.put("statsd", ImmutableMap. builder().put("port", "1234").put("prefix", "my_prefix").build()); 82 | assertFalse(new StatsReporter(new KafkaRiverConfig(new RiverSettings(globalSettings, config))).isEnabled()); 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /src/test/java/org/elasticsearch/river/kafka/StatsTest.java: -------------------------------------------------------------------------------- 1 | /* Copyright 2013 Endgame, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, 10 | * software distributed under the License is distributed on an 11 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 12 | * KIND, either express or implied. See the License for the 13 | * specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | package org.elasticsearch.river.kafka; 17 | 18 | import junit.framework.TestCase; 19 | 20 | public class StatsTest extends TestCase { 21 | 22 | public void testIt() 23 | { 24 | Stats s = new Stats(); 25 | 26 | assertEquals(0, s.failed); 27 | assertEquals(0, s.succeeded); 28 | assertEquals(0, s.flushes); 29 | assertEquals(0, s.numMessages); 30 | assertEquals(0.0, s.rate); 31 | 32 | s.failed = 50; 33 | s.succeeded = 50; 34 | s.flushes = 50; 35 | s.numMessages = 50; 36 | s.rate = 50.0; 37 | 38 | s.reset(); 39 | 40 | assertEquals(0, s.failed); 41 | assertEquals(0, s.succeeded); 42 | assertEquals(0, s.flushes); 43 | assertEquals(0, s.numMessages); 44 | assertEquals(0.0, s.rate); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=INFO, out 2 | 3 | log4j.appender.out=org.apache.log4j.ConsoleAppender 4 | log4j.appender.out.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.out.layout.conversionPattern=[%d{ISO8601}][%-5p][%-25c] %m%n 6 | --------------------------------------------------------------------------------