├── .gitignore
├── Makefile
├── README.md
├── kafka-spraynozzle.sh
├── pom.xml
└── src
├── main
├── java
│ └── com
│ │ └── uber
│ │ └── kafkaSpraynozzle
│ │ ├── KafkaArgs.java
│ │ ├── KafkaFilter.java
│ │ ├── KafkaNoopFilter.java
│ │ ├── KafkaPoster.java
│ │ ├── KafkaReader.java
│ │ ├── KafkaSpraynozzle.java
│ │ ├── ListSerializer.java
│ │ ├── SpraynozzleLeaderLatch.java
│ │ ├── benchmark
│ │ ├── BenchmarkArgs.java
│ │ ├── BenchmarkSpraynozzle.java
│ │ ├── KafkaPosterSimulator.java
│ │ └── KafkaReaderDriver.java
│ │ └── stats
│ │ ├── NoopReporterFactory.java
│ │ ├── StatsReporterFactory.java
│ │ └── StatsdReporterFactory.java
└── node
│ └── test-server.js
└── test
└── java
└── com
└── uber
└── kafkaSpraynozzle
└── ListSerializerTest.java
/.gitignore:
--------------------------------------------------------------------------------
1 | # Eclipse #
2 | ###########
3 | *target*
4 | *.jar
5 | *.war
6 | *.ear
7 | *.class
8 |
9 | # eclipse specific git ignore
10 | *.pydevproject
11 | .project
12 | .metadata
13 | bin/**
14 | tmp/**
15 | tmp/**/*
16 | *.tmp
17 | *.bak
18 | *.swp
19 | *~.nib
20 | local.properties
21 | .classpath
22 | .settings/
23 | .loadpath
24 |
25 | # External tool builders
26 | .externalToolBuilders/
27 |
28 | # Locally stored "Eclipse launch configurations"
29 | *.launch
30 |
31 | # Compiled source #
32 | ###################
33 | *.com
34 | *.class
35 | *.dll
36 | *.exe
37 | *.a
38 | *.o
39 | *.so
40 | *.node
41 | bin/*
42 |
43 | # Packages #
44 | ############
45 | # it's better to unpack these files and commit the raw source
46 | # git has its own built in compression methods
47 | *.7z
48 | *.dmg
49 | *.gz
50 | *.iso
51 | *.jar
52 | *.rar
53 | *.tar
54 | *.zip
55 |
56 | # Logs and databases #
57 | ######################
58 | *.log
59 | dump.rdb
60 | *.tap
61 |
62 | # OS generated files #
63 | ######################
64 | .DS_Store?
65 | .DS_Store
66 | ehthumbs.db
67 | Icon?
68 | Thumbs.db
69 |
70 | # Text Editor Byproducts #
71 | ##########################
72 | *.swp
73 | *.swo
74 | .idea/
75 | *~
76 |
77 | # IntelliJ Project File #
78 | #########################
79 | *.iml
80 |
81 | # Node.js stuff #
82 | #################
83 | node_modules/
84 |
--------------------------------------------------------------------------------
/Makefile:
--------------------------------------------------------------------------------
1 | .PHONY: build clean
2 |
3 | build:
4 | mvn clean package
5 |
6 | clean:
7 | mvn clean
8 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # kafka-spraynozzle
2 |
3 | (This project is deprecated and not maintained.)
4 |
5 | A nozzle to spray a kafka topic at an HTTP endpoint.
6 |
7 | ## Why?
8 |
9 | Intended to be used in combination with a load-balancing service such as HAProxy, NginX, or one built into a language like Node Cluster.
10 |
11 | Most non-Java Kafka clients suck, especially the Kafka 0.7 clients, so they can't handle the load of a truly high throughput kafka stream. This project trades the reachability guarantees of Kafka for a protocol that essentially every language out there has an optimized codebase for: HTTP.
12 |
13 | *DO NOT* use this if you expect any of the following from Kafka:
14 |
15 | * Messages are always delivered to the client because the client asks for the data upfront.
16 | * Messages are not lost during a service restart.
17 | * Older messages can be re-queried for if later analysis deems them important for a second, more detailed pass.
18 | * You can't scale your service horizontally any further and need a vertical solution.
19 |
20 | *DO* use this if any of the following about Kafka has impacted you:
21 |
22 | * Your Kafka topics have become so high throughput that you need multipler workers, so you first have your master process just consume and feed data to child workers.
23 | * Then that's not enough, so you throw away the Kafka aggregate concept and couple your consumer's internals to your producer's internals and feed directly on the producer servers.
24 | * Then that's not enough, so you couple the producers and consumers more by breaking the producer's topic into many topics. Maybe you reintroduce the aggregate, maybe you don't.
25 | * You huddle in fear when the most logically segmented kafka topics become too burdensome for the consumer to bear.
26 |
27 | ## Usage
28 |
29 | Also make sure you have `maven` installed. It's used to build the project. Optionally have any version of Node.js installed if you want to run the trivial test server that was used in debugging the spraynozzle.
30 |
31 | Finally, you can do:
32 |
33 | ```sh
34 | git clone git@github.com:uber/kafka-spraynozzle.git
35 | cd kafka-spraynozzle
36 | mvn clean package
37 | ./kafka-spraynozzle.sh -n num_http_threads -u http://remote.server:port/url -z zookeeper.server:port kafka.topic
38 | ```
39 |
40 | ## How It Works
41 |
42 | Threads! Wonderful, magical threads!
43 |
44 | 
45 |
46 | More seriously, the spraynozzle has two types of worker threads: kafka partition consumer threads and http posting threads, with a thread-safe queue created on the main thread gluing them together. The consumer threads read from their designated partition as fast as possible and unwrap the message from kafka's format and rewrap it into a format the http clients can understand, then push it onto the queue. The posting threads read from the top of the queue, construct a post request, stuff in the message, and then read and immediately throw away the response (to keep the http socket alive and not consume all available sockets on the box).
47 |
48 | This aggregation and re-dissemination seems silly when you stop to think about it, but what it's actually doing is working around a design flaw in Kafka: the producer has to know the maximum amount of data it can throw into a single partition based on however much any individual consuming worker can handle. This is a design flaw because:
49 |
50 | 1. You can't predict that ahead of time.
51 | 2. Even if you do figure it out for one consumer, it'll be different for another even if its written in the same language because the amount of work it does on the data differs.
52 |
53 | So kafka-spraynozzle actually has *three* knobs to adjust involving data flow:
54 |
55 | 1. The number of partitions to consume. That's easy, just set it to the partition count of the topic at hand.
56 | 2. The number of http threads to run. This one depends on the response time of your workers and the messages/sec of your topic. 1/response time gives you the messages/sec of each http worker, so messages/sec/response time gives you the number of http threads to run.
57 | 3. The number of workers sitting behind HAProxy or NginX or Node Cluster or whatever. This is also relatively simple: the number of messages/sec of your topic divided by the number of messages/sec each worker can *safely* consume (there should be some buffer for spikes in the stream to not choke your workers).
58 |
59 | ## License (MIT)
60 |
61 | Copyright (C) 2014 by Uber Technologies, Inc
62 |
63 | Permission is hereby granted, free of charge, to any person obtaining a copy
64 | of this software and associated documentation files (the "Software"), to deal
65 | in the Software without restriction, including without limitation the rights
66 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
67 | copies of the Software, and to permit persons to whom the Software is
68 | furnished to do so, subject to the following conditions:
69 |
70 | The above copyright notice and this permission notice shall be included in
71 | all copies or substantial portions of the Software.
72 |
73 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
74 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
75 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
76 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
77 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
78 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
79 | THE SOFTWARE.
80 |
--------------------------------------------------------------------------------
/kafka-spraynozzle.sh:
--------------------------------------------------------------------------------
1 | exec -a kafka-spraynozzle-${@: -1} java -classpath ./target/kafkaSpraynozzle-0.1.0.jar com.uber.kafkaSpraynozzle.KafkaSpraynozzle "$@"
2 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
3 | 4.0.0
4 | com.uber
5 | kafkaSpraynozzle
6 | jar
7 | 0.1.0
8 | Kafka Spraynozzle
9 |
10 | 3.1.1
11 |
12 |
13 |
14 | readytalk-maven
15 | https://dl.bintray.com/readytalk/maven/
16 |
17 |
18 |
19 |
20 | junit
21 | junit
22 | 4.12
23 | test
24 |
25 |
26 | org.apache.kafka
27 | kafka
28 | 0.7.2
29 |
30 |
31 | com.101tec
32 | zkclient
33 | 0.5
34 |
35 |
36 | org.apache.httpcomponents
37 | httpclient
38 | 4.3.6
39 |
40 |
41 | com.lexicalscope.jewelcli
42 | jewelcli
43 | 0.8.7
44 |
45 |
46 | com.fasterxml.jackson.core
47 | jackson-core
48 | 2.5.3
49 |
50 |
51 | com.fasterxml.jackson.core
52 | jackson-databind
53 | 2.5.3
54 |
55 |
56 | com.fasterxml.jackson.core
57 | jackson-annotations
58 | 2.5.3
59 |
60 |
61 | com.timgroup
62 | java-statsd-client
63 | 3.1.0
64 |
65 |
66 | org.apache.curator
67 | curator-framework
68 | 2.8.0
69 |
70 |
71 | org.apache.curator
72 | curator-recipes
73 | 2.8.0
74 |
75 |
76 | org.scala-lang
77 | scala-library
78 | 2.8.0
79 |
80 |
81 | com.readytalk
82 | metrics3-statsd
83 | 4.1.0
84 |
85 |
86 | io.dropwizard.metrics
87 | metrics-core
88 | ${io.dropwizard.metrics.version}
89 |
90 |
91 | io.dropwizard.metrics
92 | metrics-jvm
93 | ${io.dropwizard.metrics.version}
94 |
95 |
96 | com.github.tomakehurst
97 | wiremock
98 | 1.57
99 |
100 |
101 |
102 |
103 |
104 | org.apache.maven.plugins
105 | maven-shade-plugin
106 | 1.6
107 |
108 | true
109 |
110 |
111 | *:*
112 |
113 | META-INF/*.SF
114 | META-INF/*.DSA
115 | META-INF/*.RSA
116 |
117 |
118 |
119 |
120 |
121 |
122 | package
123 |
124 | shade
125 |
126 |
127 |
128 |
129 |
130 | com.uber.kafkaSpraynozzle.KafkaSpraynozzle
131 |
132 |
133 |
134 |
135 |
136 |
137 |
138 | org.apache.maven.plugins
139 | maven-compiler-plugin
140 | 3.2
141 |
142 | 1.7
143 | 1.7
144 | 1.7
145 | 1.7
146 |
147 | -Xlint:unchecked
148 |
149 |
150 |
151 |
152 | org.codehaus.mojo
153 | cobertura-maven-plugin
154 | 2.6
155 |
156 |
157 |
158 | com.uber.kafkaSpraynozzle.*$$*
159 |
160 |
161 | com/uber/kafkaSpraynozzle/*$$*
162 |
163 |
164 |
165 |
166 |
167 |
168 |
169 |
170 |
171 | org.codehaus.mojo
172 | cobertura-maven-plugin
173 | 2.6
174 |
175 |
176 | html
177 | xml
178 |
179 |
180 |
181 | com.uber.kafkaSpraynozzle.*$$*
182 |
183 |
184 | com/uber/kafkaSpraynozzle/*$$*
185 |
186 |
187 |
188 |
189 |
190 | org.apache.maven.plugins
191 | maven-javadoc-plugin
192 | 2.10.1
193 |
194 |
195 | org.apache.maven.plugins
196 | maven-resources-plugin
197 | 2.7
198 |
199 |
200 | org.codehaus.mojo
201 | findbugs-maven-plugin
202 | 3.0.0
203 |
204 |
205 |
206 |
207 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaArgs.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import com.lexicalscope.jewel.cli.CommandLineInterface;
4 | import com.lexicalscope.jewel.cli.Option;
5 | import com.lexicalscope.jewel.cli.Unparsed;
6 | import java.util.List;
7 |
8 | @CommandLineInterface(application="kafka-spraynozzle")
9 | public interface KafkaArgs {
10 | @Unparsed(name="topic", description="The kafka topic to stream from") String getTopic();
11 | @Option(shortName="u", description="The URL(s) to post kafka messages to (in the form http(s)://server.address(:port)/url)") List getUrls();
12 | @Option(shortName="z", description="The Zookeeper instance to read from (in the form server.address:port)") String getZk();
13 | @Option(shortName="n", description="The number of HTTP posting threads to use") int getThreadCount();
14 | @Option(shortName="b", description="Use Kafka protocol to buffer messages while spraynozzle is down (default: false)") boolean getBuffering();
15 | @Option(shortName="p", defaultValue="1", description="The number of topic partitions (default: 1)") int getPartitionCount();
16 | @Option(shortName="f", defaultToNull=true, description="The name of a class to use for filtering messages (default: none)") String getFilterClass();
17 | @Option(shortName="l", defaultToNull=false, description="Use zk to make the spraynozzle highly available (default: false)") Boolean getIsHighlyAvailable();
18 | @Option(shortName="c", defaultToNull=true, description="The classpath to use for finding the above-mentioned filter class (default: none)") String getFilterClasspath();
19 | @Option(shortName="a", defaultToNull=true, description="The arguments for the above-mentioned filter class (default: none)") String getFilterClassArgs();
20 | @Option(shortName="s", defaultToNull=true, description="The name of a class to use for stats reporting (default: none)") String getStatsClass();
21 | @Option(shortName="w", defaultToNull=true, description="The classpath to use for finding the above-mentioned stats class (default: none)") String getStatsClasspath();
22 | @Option(shortName="d", defaultToNull=true, description="The arguments for the above-mentioned stats class (default: none)") String getStatsClassArgs();
23 | @Option(longName="connectionTimeout", defaultToNull=true, description="The connection timeout in milliseconds for posting (default: none)") Integer getConnectionTimeout();
24 | @Option(longName="socketTimeout", shortName="t", defaultToNull=true, description="The socket timeout in milliseconds for posting (default: none)") Integer getSocketTimeout();
25 | @Option(longName="help", shortName="h", helpRequest=true, description="Display this Help message") boolean getHelp();
26 | @Option(shortName="i", defaultValue = "1", description="Number of messages to pack as a batch before POST to endpoint") int getBatchSize();
27 | @Option(shortName="e", description="Enable Least-Response-Time load balancing when posting") boolean getEnableBalancing();
28 | }
29 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaFilter.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import org.apache.http.entity.ByteArrayEntity;
4 |
5 | public interface KafkaFilter {
6 | /**
7 | * Filter the incoming entity, returning the data to pass on or null to skip
8 | * @param jsonEntity the entity to potentially filter
9 | * @return the filtered data or null to skip
10 | */
11 | public ByteArrayEntity filter(ByteArrayEntity jsonEntity);
12 | }
13 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaNoopFilter.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import org.apache.http.entity.ByteArrayEntity;
4 |
5 | public class KafkaNoopFilter implements KafkaFilter {
6 | public KafkaNoopFilter() { }
7 | public ByteArrayEntity filter(ByteArrayEntity jsonEntity) {
8 | return jsonEntity;
9 | }
10 | }
11 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaPoster.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import java.io.IOException;
4 | import java.util.ArrayList;
5 | import java.util.Date;
6 | import java.util.List;
7 | import java.util.concurrent.ConcurrentLinkedQueue;
8 |
9 | import com.codahale.metrics.Counter;
10 | import com.codahale.metrics.MetricRegistry;
11 | import com.codahale.metrics.Timer;
12 | import org.apache.http.client.config.RequestConfig;
13 | import kafka.common.InvalidMessageSizeException;
14 | import org.apache.http.entity.ByteArrayEntity;
15 | import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
16 | import org.apache.http.impl.client.HttpClientBuilder;
17 | import org.apache.http.impl.client.CloseableHttpClient;
18 | import org.apache.http.client.methods.CloseableHttpResponse;
19 | import org.apache.http.client.methods.HttpPost;
20 | import org.apache.http.util.EntityUtils;
21 |
22 | public class KafkaPoster implements Runnable {
23 | private MetricRegistry metricRegistry;
24 | ConcurrentLinkedQueue queue;
25 | PoolingHttpClientConnectionManager cm;
26 | List urls;
27 | int currentUrl = 0;
28 | static final int responseDecayingHalflife = 5;
29 | KafkaFilter messageFilter;
30 | private RequestConfig requestConfig;
31 | private boolean roundRobin = false;
32 | private int batchSize = 20;
33 | private int batchMaxBytes = 128*1024;
34 | private boolean debuggingOutput = false;
35 | private Timer postTime = new Timer();
36 | private Counter postCount = new Counter();
37 | private Counter postSuccess = new Counter();
38 | private Counter postFailure = new Counter();
39 | private Counter filteredCount = new Counter();
40 |
41 | public KafkaPoster(
42 | MetricRegistry metricRegistry,
43 | ConcurrentLinkedQueue queue,
44 | PoolingHttpClientConnectionManager cm,
45 | List urls,
46 | KafkaFilter messageFilter,
47 | Integer socketTimeout,
48 | Integer connectionTimeout
49 | ){
50 | this(metricRegistry,
51 | queue, cm, urls, messageFilter,
52 | socketTimeout, connectionTimeout,
53 | 1, false, false );
54 | }
55 | public KafkaPoster(
56 | MetricRegistry metricRegistry,
57 | ConcurrentLinkedQueue queue,
58 | PoolingHttpClientConnectionManager cm,
59 | List urls,
60 | KafkaFilter messageFilter,
61 | Integer socketTimeout,
62 | Integer connectionTimeout,
63 | int batchSize,
64 | boolean roundRobinPost,
65 | boolean debuggingOutput) {
66 | this.metricRegistry = metricRegistry;
67 | this.queue = queue;
68 | this.cm = cm;
69 | this.urls = urls;
70 | this.messageFilter = messageFilter;
71 | RequestConfig.Builder builder = RequestConfig.custom();
72 | if (socketTimeout != null) {
73 | builder = builder.setSocketTimeout(socketTimeout);
74 | }
75 | if (connectionTimeout != null) {
76 | builder = builder.setConnectTimeout(connectionTimeout);
77 | }
78 | this.requestConfig = builder.build();
79 |
80 | if (metricRegistry != null) {
81 | this.postTime = metricRegistry.timer("post_time");
82 | this.postCount = metricRegistry.counter("post_count");
83 | this.postSuccess = metricRegistry.counter("post_success");
84 | this.postFailure = metricRegistry.counter("post_failure");
85 | this.filteredCount = metricRegistry.counter("filtered_count");
86 | }
87 | if (batchSize <= 0){
88 | batchSize = 1;
89 | }
90 | this.batchSize = batchSize;
91 | this.roundRobin = roundRobinPost;
92 | this.debuggingOutput = debuggingOutput;
93 | }
94 |
95 | private int leastResponseIdx(long[] responseTime) {
96 | int leastIdx = 0;
97 | for (int i = 1; i < responseTime.length; ++i){
98 | if (responseTime[i] < responseTime[leastIdx]){
99 | leastIdx = i;
100 | }
101 | }
102 | return leastIdx;
103 | }
104 |
105 | private int getBatchDataSize(List batch){
106 | int totalSize = 0;
107 | for (int i = 0; i < batch.size(); ++i){
108 | totalSize += (int)batch.get(i).getContentLength();
109 | }
110 | return totalSize;
111 | }
112 |
113 | private CloseableHttpClient client = null;
114 | private long threadId = 0L;
115 | private long lastReconnect = 0L;
116 | private long[] responseTime = null;
117 | private long[] responseTimestamp = null;
118 | public void run() {
119 | threadId = Thread.currentThread().getId();
120 | System.out.println("Starting poster thread " + threadId);
121 | client = HttpClientBuilder.create().setConnectionManager(cm).build();
122 | lastReconnect = new Date().getTime();
123 | responseTime = new long[urls.size()];
124 | responseTimestamp = new long[urls.size()];
125 |
126 | List batch = new ArrayList(batchSize + 1);
127 |
128 | long totalPostingTimeMs = 0L;
129 | long totalPostingCount = 0L;
130 | long lastReportingTime = new Date().getTime();
131 | while(true) {
132 | ByteArrayEntity jsonEntity = queue.poll();
133 | if(jsonEntity != null) {
134 | jsonEntity = messageFilter.filter(jsonEntity);
135 | if (jsonEntity != null) {
136 | batch.add(jsonEntity);
137 |
138 | if (batch.size() >= batchSize || getBatchDataSize(batch) >= batchMaxBytes) {
139 | long timeBeforePost = new Date().getTime();
140 | if (debuggingOutput && timeBeforePost - lastReportingTime > 5*1000) {
141 | System.out.println(String.format(
142 | "Posting thread %d, avg posting cost %d ms, total posting called %d",
143 | threadId, totalPostingTimeMs / totalPostingCount, totalPostingCount));
144 | lastReportingTime = timeBeforePost;
145 | }
146 | postBatchEvents(batch);
147 | totalPostingCount += 1;
148 | totalPostingTimeMs += (new Date().getTime() - timeBeforePost);
149 | }
150 | } else {
151 | filteredCount.inc();
152 | }
153 | } else {
154 | postBatchEvents(batch);
155 | try {
156 | Thread.sleep(250);
157 | } catch (java.lang.InterruptedException e) {
158 | System.out.println("Sleep issue!?");
159 | e.printStackTrace();
160 | }
161 | }
162 | }
163 | }
164 |
165 | private boolean postBatchEvents(List batch) {
166 | if (batch.isEmpty()) {
167 | return true;
168 | }
169 |
170 | final long NANOS_PER_SECOND = 1000L * 1000L * 1000L;
171 | final long NANOS_PER_MILLI_SECOND = 1000L * 1000L;
172 | int pickedUrlIdx = currentUrl;
173 | try (Timer.Context ctx = postTime.time()) {
174 | long timeBeforePost = System.nanoTime();
175 | postCount.inc();
176 | if (roundRobin) {
177 | pickedUrlIdx = currentUrl;
178 | } else {
179 | pickedUrlIdx = leastResponseIdx(responseTime);
180 | }
181 | HttpPost post = new HttpPost(urls.get(pickedUrlIdx));
182 | if (roundRobin) {
183 | currentUrl = (currentUrl + 1) % urls.size();
184 | }
185 | post.setHeader("User-Agent", "KafkaSpraynozzle-0.2.0");
186 |
187 | // we set the timestamp and time cost as if it has failed
188 | // So in case it fails we will avoid it in next try;
189 | // Later if we succeeded we will update the value to correct latest value.
190 | responseTimestamp[pickedUrlIdx] = System.nanoTime();
191 | responseTime[pickedUrlIdx] = 5 * NANOS_PER_SECOND;
192 |
193 |
194 | if (batchSize == 1 && batch.size() == 1){
195 | post.setEntity(batch.get(0));
196 | batch.clear();
197 | }else {
198 | try {
199 | ByteArrayEntity packedEntities = ListSerializer.toByteArrayEntity(batch);
200 | post.setEntity(packedEntities);
201 | batch.clear();
202 | } catch (InvalidMessageSizeException ex) {
203 | postFailure.inc();
204 | batch.clear();
205 | return false;
206 | }
207 | }
208 | CloseableHttpResponse response = client.execute(post);
209 | int statusCode = response.getStatusLine().getStatusCode();
210 | long timeAfterPost = System.nanoTime();
211 | if (statusCode >= 200 && statusCode < 300) {
212 | postSuccess.inc();
213 | //// managing the "least response time" decay.
214 | //// Every time halflife time has passed, we reduce the "bad record" of response time by half
215 | /// so eventually those bad servers with slow response time will get another chance of being tried.
216 | /// and we will not keep pounding the same fast server since all record of response time decays.
217 | responseTimestamp[pickedUrlIdx] = timeAfterPost;
218 | responseTime[pickedUrlIdx] = (timeAfterPost - timeBeforePost) + 2 * NANOS_PER_MILLI_SECOND;// penalize by 2 ms so the same won't be used again and again.
219 | } else {
220 | postFailure.inc();
221 | responseTimestamp[pickedUrlIdx] = timeAfterPost;
222 | responseTime[pickedUrlIdx] = (timeAfterPost - timeBeforePost) + 5 * NANOS_PER_SECOND;// failed posts gets 5 second penalty
223 | }
224 |
225 |
226 | for (int i = 0; i < responseTimestamp.length; ++i) {
227 | if ((timeAfterPost - responseTimestamp[i]) > responseDecayingHalflife * NANOS_PER_SECOND) {
228 | responseTimestamp[i] = timeAfterPost;
229 | responseTime[i] = responseTime[i] / 2;
230 | }
231 | }
232 | long currentTime = new Date().getTime();
233 | if (currentTime - lastReconnect > 10000) {
234 | lastReconnect = currentTime;
235 | response.close();
236 | } else {
237 | EntityUtils.consume(response.getEntity());
238 | }
239 | } catch (IOException e) {
240 | System.out.println("IO issue");
241 | e.printStackTrace();
242 | postFailure.inc();
243 | responseTimestamp[pickedUrlIdx] = System.nanoTime();
244 | responseTime[pickedUrlIdx] = 5 * NANOS_PER_SECOND;
245 |
246 | return false;
247 | }
248 | return true;
249 | }
250 | }
251 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaReader.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import java.nio.ByteBuffer;
4 | import java.util.concurrent.ConcurrentLinkedQueue;
5 |
6 | import com.codahale.metrics.Counter;
7 | import com.codahale.metrics.MetricRegistry;
8 | import com.codahale.metrics.Timer;
9 | import kafka.consumer.KafkaStream;
10 | import kafka.message.Message;
11 | import kafka.message.MessageAndMetadata;
12 | import org.apache.http.entity.ByteArrayEntity;
13 | import org.apache.http.entity.ContentType;
14 |
15 | public class KafkaReader implements Runnable {
16 | private MetricRegistry metricRegistry;
17 | private ConcurrentLinkedQueue queue;
18 | private KafkaStream stream;
19 |
20 | private Counter enqueuedCount = new Counter();
21 | private Counter pausedCount = new Counter();
22 | private Timer pausedTime = new Timer();
23 |
24 | public KafkaReader(MetricRegistry metricRegistry,
25 | ConcurrentLinkedQueue queue,
26 | KafkaStream stream) {
27 | this.metricRegistry = metricRegistry;
28 | this.queue = queue;
29 | this.stream = stream;
30 | if (metricRegistry != null) {
31 | this.enqueuedCount = metricRegistry.counter("enqueued_count");
32 | this.pausedCount = metricRegistry.counter("pause_count");
33 | this.pausedTime = metricRegistry.timer("pause_time");
34 | }
35 | }
36 |
37 | public void run() {
38 | long threadId = Thread.currentThread().getId();
39 | // Supposedly the HTTP Client is threadsafe, but lets not chance it, eh?
40 | System.out.println("Starting reader thread " + threadId);
41 | int pushCount = 0;
42 | for(MessageAndMetadata msgAndMetadata: this.stream) {
43 | ByteBuffer message = msgAndMetadata.message().payload();
44 | Integer messageLen = msgAndMetadata.message().payloadSize();
45 | pushCount = enqueueData(pushCount, messageLen, message);
46 | }
47 | }
48 |
49 | public int enqueueData(int pushCount, Integer messageLen, ByteBuffer message) {
50 | Integer messageOffset = message.arrayOffset();
51 | ByteArrayEntity jsonEntity = new ByteArrayEntity(message.array(), messageOffset, messageLen, ContentType.APPLICATION_JSON);
52 | jsonEntity.setContentEncoding("UTF-8");
53 | queue.add(jsonEntity);
54 | enqueuedCount.inc();
55 | pushCount++;
56 | if (pushCount == 100) {
57 | pushCount = 0;
58 | int queueSize = queue.size();
59 | if (queueSize > 500) {
60 | pausedCount.inc();
61 | try (Timer.Context ctx = pausedTime.time()) {
62 | while (queueSize > 100) {
63 | try {
64 | Thread.sleep(5);
65 | } catch (InterruptedException e) {
66 | System.out.println("Sleep issue!?");
67 | e.printStackTrace();
68 | }
69 | queueSize = queue.size();
70 | }
71 | }
72 | }
73 | }
74 | return pushCount;
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/KafkaSpraynozzle.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | // Forgive me for any non-idiomatic ways this code behaves. I'm not a Java guy but the other clients suck.
4 | import com.codahale.metrics.JvmAttributeGaugeSet;
5 | import com.codahale.metrics.MetricRegistry;
6 | import com.codahale.metrics.ScheduledReporter;
7 | import com.codahale.metrics.jvm.BufferPoolMetricSet;
8 | import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
9 | import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
10 | import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
11 | import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
12 | import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
13 | import com.lexicalscope.jewel.cli.ArgumentValidationException;
14 | import com.lexicalscope.jewel.cli.CliFactory;
15 | import com.uber.kafkaSpraynozzle.stats.NoopReporterFactory;
16 | import java.io.File;
17 | import java.lang.ClassLoader;
18 | import java.lang.ClassNotFoundException;
19 | import java.lang.IllegalAccessException;
20 | import java.lang.InstantiationException;
21 | import java.lang.management.ManagementFactory;
22 | import java.lang.reflect.InvocationTargetException;
23 | import java.net.MalformedURLException;
24 | import java.net.URL;
25 | import java.net.URLClassLoader;
26 | import java.util.ArrayList;
27 | import java.util.HashMap;
28 | import java.util.List;
29 | import java.util.Map;
30 | import java.util.Properties;
31 | import java.util.concurrent.ConcurrentLinkedQueue;
32 | import java.util.concurrent.ExecutorService;
33 | import java.util.concurrent.Executors;
34 |
35 | import com.uber.kafkaSpraynozzle.stats.StatsReporterFactory;
36 | import kafka.consumer.Consumer;
37 | import kafka.consumer.ConsumerConfig;
38 | import kafka.consumer.KafkaStream;
39 | import kafka.javaapi.consumer.ConsumerConnector;
40 | import kafka.message.Message;
41 | import kafka.utils.ZkUtils;
42 | import org.I0Itec.zkclient.ZkClient;
43 | import org.apache.http.config.ConnectionConfig;
44 | import org.apache.http.entity.ByteArrayEntity;
45 | import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
46 | import java.util.UUID;
47 | import java.util.concurrent.TimeUnit;
48 |
49 | class KafkaSpraynozzle {
50 | public static void main(String[] args) throws Exception {
51 | KafkaArgs spraynozzleArgs;
52 | try {
53 | spraynozzleArgs = CliFactory.parseArguments(KafkaArgs.class, args);
54 | } catch(ArgumentValidationException e) {
55 | System.err.println(e.getMessage());
56 | return;
57 | }
58 |
59 | String zk = spraynozzleArgs.getZk();
60 | ZkClient zkClient = new ZkClient(zk, 10000);
61 | Boolean spraynozzleHA = spraynozzleArgs.getIsHighlyAvailable();
62 | boolean buffering = spraynozzleArgs.getBuffering();
63 |
64 | String topic = spraynozzleArgs.getTopic();
65 | final List urls = spraynozzleArgs.getUrls();
66 | final String cleanedUrl = urls.get(0).replaceAll("[/\\:]", "_");
67 | final int threadCount = spraynozzleArgs.getThreadCount();
68 | final int partitionCount = spraynozzleArgs.getPartitionCount();
69 | final String filterClass = spraynozzleArgs.getFilterClass();
70 | final String filterClasspath = spraynozzleArgs.getFilterClasspath();
71 | final String filterClassArgs = spraynozzleArgs.getFilterClassArgs();
72 | final String statsClass = spraynozzleArgs.getStatsClass();
73 | final String statsClasspath = spraynozzleArgs.getStatsClasspath();
74 | final String statsClassArgs = spraynozzleArgs.getStatsClassArgs();
75 | final Integer soTimeout = spraynozzleArgs.getSocketTimeout();
76 | final Integer connectTimeout = spraynozzleArgs.getConnectionTimeout();
77 | final int batchPostingSize = spraynozzleArgs.getBatchSize();
78 | final boolean forceRoundRobin = !spraynozzleArgs.getEnableBalancing();
79 | String[] topics = topic.split(",");
80 | if (topics.length == 1) {
81 | System.out.println("Listening to " + topic + " topic from " + zk + " and redirecting to " + urls);
82 | } else {
83 | System.out.println("Listening to " + topic + " topics from " + zk + " and redirecting to " + urls);
84 | }
85 |
86 | // IMPORTANT: It is highly recommended to turn on spraynozzleHA and buffering sumultaneously
87 | // so messages are not dropped in the leader election process
88 | if (spraynozzleHA) {
89 | String zkLeaderLatchFolderPath = "/consumers/kafka_spraynozzle_leader_latch_" + topics[0] + cleanedUrl;
90 | System.out.println("Performing leader election through zookeeper and picking leader that will proceed.");
91 | //use same zk as kafka
92 | //identify each spraynozzle instace with a UUID to allow spraynozzles in the same ring (master-slave config) to coexist in the same host
93 | String spraynozzleName = "spraynozzle-" + UUID.randomUUID();
94 | SpraynozzleLeaderLatch curatorClient = new SpraynozzleLeaderLatch(zk, zkLeaderLatchFolderPath, spraynozzleName);
95 | curatorClient.start();
96 | curatorClient.blockUntilisLeader();
97 | System.out.println("This spraynozzle (" + spraynozzleName + ") is now the leader. Follow the leader!");
98 | }
99 |
100 | if (!buffering) {
101 | // Clear out zookeeper records so the spraynozzle drops messages between runs
102 | clearZkPath(zkClient, "/consumers/kafka_spraynozzle_" + topics[0] + cleanedUrl);
103 | }
104 |
105 | // Kafka setup stuff
106 | Properties kafkaProps = new Properties();
107 | kafkaProps.put("zk.connect", zk);
108 | kafkaProps.put("zk.connectiontimeout.ms", "10000");
109 | kafkaProps.put("groupid", "kafka_spraynozzle_" + topics[0] + cleanedUrl);
110 | kafkaProps.put("autooffset.reset", "largest");
111 | kafkaProps.put("fetch.size", String.valueOf(2*1024*1024));
112 | ConsumerConfig consumerConfig = new ConsumerConfig(kafkaProps);
113 | ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(consumerConfig);
114 | HashMap topicParallelism = new HashMap();
115 | for (int i = 0; i < topics.length; i++) {
116 | topicParallelism.put(topics[i], partitionCount);
117 | }
118 | Map>> topicMessageStreams = consumerConnector.createMessageStreams(topicParallelism);
119 | ArrayList>> streams = new ArrayList>>();
120 | for (int i = 0; i < topics.length; i++) {
121 | List> stream = topicMessageStreams.get(topics[i]);
122 | streams.add(stream);
123 | }
124 | ExecutorService executor = Executors.newFixedThreadPool(threadCount + (partitionCount * topics.length) + 1);
125 |
126 | // Http Connection Pooling stuff
127 | final PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager();
128 | cm.setMaxTotal(threadCount);
129 | cm.setDefaultMaxPerRoute(threadCount / urls.size());
130 |
131 | // Message-passing queues within the spraynozzle
132 | final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue();
133 |
134 | // Build the worker threads
135 | MetricRegistry metricRegistry = new MetricRegistry();
136 | StatsReporterFactory statsReporter = getStatsReporter(statsClasspath, statsClass, statsClassArgs);
137 | if (statsReporter != null) {
138 | ScheduledReporter reporter = statsReporter.build(metricRegistry);
139 | if (reporter != null) {
140 | System.out.println("Starting stats reporter");
141 | metricRegistry.register("jvm.attribute", new JvmAttributeGaugeSet());
142 | metricRegistry.register("jvm.buffers", new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer()));
143 | metricRegistry.register("jvm.classloader", new ClassLoadingGaugeSet());
144 | metricRegistry.register("jvm.filedescriptor", new FileDescriptorRatioGauge());
145 | metricRegistry.register("jvm.gc", new GarbageCollectorMetricSet());
146 | metricRegistry.register("jvm.memory", new MemoryUsageGaugeSet());
147 | metricRegistry.register("jvm.threads", new ThreadStatesGaugeSet());
148 | reporter.start(10, TimeUnit.SECONDS);
149 | }
150 | }
151 |
152 | for (final List> streamList : streams) {
153 | for (final KafkaStream stream : streamList) {
154 | executor.submit(new KafkaReader(metricRegistry, queue, stream));
155 | }
156 | }
157 |
158 | for (int i = 0; i < threadCount; i++) {
159 | // create new filter for every thread so the filters member variables are not shared
160 | KafkaFilter messageFilter = getKafkaFilter(filterClass, filterClasspath, filterClassArgs);
161 | executor.submit(new KafkaPoster(metricRegistry, queue, cm, urls, messageFilter, soTimeout, connectTimeout,
162 | batchPostingSize, forceRoundRobin, false));
163 | }
164 | }
165 |
166 | private static void clearZkPath(ZkClient zkClient, String zkPath){
167 | ZkUtils.deletePathRecursive(zkClient, zkPath);
168 | while (ZkUtils.pathExists(zkClient, zkPath)) {
169 | try {
170 | Thread.sleep(250);
171 | } catch (java.lang.InterruptedException e) {
172 | System.out.println("Sleep Exception!?");
173 | e.printStackTrace();
174 | }
175 | }
176 | }
177 |
178 | private static KafkaFilter getKafkaFilter(String filterClass, String filterClasspath, String filterClassArgs){
179 | KafkaFilter messageFilter = (KafkaFilter)getClass(filterClasspath, filterClass, filterClassArgs);
180 | if (messageFilter == null) {
181 | messageFilter = new KafkaNoopFilter();
182 | }
183 | System.out.println("Using message filter: " + messageFilter);
184 | return messageFilter;
185 | }
186 |
187 | private static StatsReporterFactory getStatsReporter(String classpath, String className, String classArgs){
188 | StatsReporterFactory statsReporter = (StatsReporterFactory)getClass(classpath, className, classArgs);
189 | if (statsReporter == null) {
190 | statsReporter = new NoopReporterFactory();
191 | }
192 | System.out.println("Using stats reporter: " + statsReporter);
193 | return statsReporter;
194 | }
195 |
196 | private static Object getClass(String classpath, String className, String classArgs){
197 | Object newClass = null;
198 | if (classpath != null && className != null) {
199 | File file = new File(classpath);
200 | try {
201 | URL[] urls = new URL[]{file.toURL()};
202 | ClassLoader cl = new URLClassLoader(urls);
203 | Class> cls = cl.loadClass(className);
204 | if (classArgs == null) {
205 | newClass = cls.newInstance();
206 | } else {
207 | newClass = cls.getConstructor(String.class).newInstance(classArgs);
208 | }
209 | } catch (MalformedURLException e) {
210 | System.out.println("Bad classpath provided: " + classpath);
211 | } catch (ClassNotFoundException e) {
212 | System.out.println("Class not found: " + className);
213 | } catch (InstantiationException e) {
214 | System.out.println("Cannot create instance of class: " + className);
215 | } catch (IllegalAccessException e) {
216 | System.out.println("Class did I have no idea what but its bad and illegal: " + className);
217 | } catch (NoSuchMethodException e) {
218 | System.out.println("Class with custom arguments must have a constructor taking 1 String: " + className);
219 | } catch (InvocationTargetException e) {
220 | System.out.println("Error initializing custom class: " + className + " " + e.getMessage());
221 | } catch (Exception e) {
222 | System.out.println("Unknown error initializing custom class: " + className + " " + e.getMessage());
223 | }
224 | }
225 | return newClass;
226 | }
227 | }
228 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/ListSerializer.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import kafka.common.InvalidMessageSizeException;
4 | import org.apache.http.entity.ByteArrayEntity;
5 |
6 | import java.io.*;
7 | import java.util.ArrayList;
8 | import java.util.List;
9 | /**
10 | * Created by xdong on 11/17/15.
11 | * This serializer just serializes a list of ByteArrayEntities into one single ByteArrayEntities
12 | * Later it can be deserialized back.
13 | */
14 | public class ListSerializer {
15 |
16 | /**
17 | * Serialize the List\ object into an byte array so it can be posted in one call
18 | * The encoding rule is plain simple:
19 | * {totalElements}{length}{byte block of length}{length}{byte block of length}...
20 | * For example, a List of 3 ByteArrayEntities, with 10,20,30 bytes at length will deserialized into below:
21 | * 0x00000003, 0x00000001,[10 byte],0x00000002,[20 byte],0x00000003,[30 byte]
22 | * @param entities the List objects to serialize
23 | * @return the one single byte array, encoded.
24 | * @throws InvalidMessageSizeException If the array is too large to fit into one huge memory
25 | * @throws IOException if anything is wrong.
26 | */
27 | public static ByteArrayEntity toByteArrayEntity(List entities) throws InvalidMessageSizeException, IOException {
28 | long totalLength = 0L;
29 | final long largest_block = 1024L*1024L*128L;
30 | for (int i = 0; i < entities.size(); ++i){
31 | totalLength += entities.get(i).getContentLength();
32 | }
33 | if (totalLength >= largest_block){ // >=128MB is dangerous
34 | throw new InvalidMessageSizeException("The batched messages are too large");
35 | }
36 | ByteArrayOutputStream serialized = new ByteArrayOutputStream();
37 | DataOutputStream output = new DataOutputStream(serialized);
38 | output.writeInt(entities.size());
39 | for (ByteArrayEntity entity : entities){
40 | output.writeInt((int) entity.getContentLength());
41 | entity.writeTo(output);
42 | }
43 | return new ByteArrayEntity(serialized.toByteArray());
44 | }
45 |
46 | /**
47 | * Deserialize the byte block back into our List object
48 | * @param serialized the byte block
49 | * @return the deserialized object
50 | * @throws InvalidMessageSizeException if the data is corrupted, causing too huge data
51 | * @throws IOException if somehow stream read/write fails, or data corrupted
52 | */
53 | public static List toListOfByteArrayEntity(byte[] serialized) throws InvalidMessageSizeException, IOException {
54 | ByteArrayInputStream byteStream = new ByteArrayInputStream(serialized);
55 | DataInputStream input = new DataInputStream(byteStream);
56 | final long largest_block = 1024L*1024L*128L;
57 |
58 | int arraySize = input.readInt();
59 | List retVal = new ArrayList(arraySize);
60 | for (int i = 0; i < arraySize; ++i){
61 | int objectSize = input.readInt();
62 | if (objectSize >= largest_block || objectSize < 0){ // >=128MB is dangerous
63 | throw new InvalidMessageSizeException("The batched messages are too large");
64 | }
65 | byte[] objectBytes = new byte[objectSize];
66 | int actualRead = input.read(objectBytes, 0, objectSize);
67 | if (actualRead != objectSize){
68 | throw new IOException("Deserialized entity corrupted");
69 | }
70 | ByteArrayEntity entity = new ByteArrayEntity(objectBytes);
71 | retVal.add(entity);
72 | }
73 | return retVal;
74 | }
75 |
76 | /**
77 | * Helper class for converting ByteArrayEntity to byte[] array
78 | * @param entity The byte array entity
79 | * @return The byte array copy
80 | */
81 | public static byte[] getByteArrayEntityBytes(ByteArrayEntity entity){
82 | ByteArrayOutputStream serialized = new ByteArrayOutputStream();
83 | try {
84 | entity.writeTo(serialized);
85 | return serialized.toByteArray();
86 | }catch(IOException ex){
87 | return new byte[0];
88 | }
89 | }
90 |
91 | }
92 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/SpraynozzleLeaderLatch.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle;
2 |
3 | import org.apache.curator.framework.recipes.leader.LeaderLatch;
4 | import org.apache.curator.framework.CuratorFramework;
5 | import org.apache.curator.framework.CuratorFrameworkFactory;
6 | import org.apache.curator.retry.ExponentialBackoffRetry;
7 | import org.apache.curator.framework.recipes.leader.Participant;
8 | import org.apache.curator.framework.recipes.atomic.DistributedAtomicInteger;
9 | import org.apache.curator.framework.recipes.atomic.AtomicValue;
10 |
11 | public class SpraynozzleLeaderLatch {
12 | private CuratorFramework zkClient;
13 | private String latchPath;
14 | private String counterPath;
15 | private String id;
16 | private LeaderLatch leaderLatch;
17 |
18 | public SpraynozzleLeaderLatch(String connString, String latchPath, String id) {
19 | zkClient = CuratorFrameworkFactory.newClient(connString, new ExponentialBackoffRetry(1000, 3));
20 | this.id = id;
21 | this.latchPath = latchPath;
22 | this.counterPath = counterPath;
23 | }
24 |
25 | public void start() throws Exception {
26 | zkClient.start();
27 | zkClient.blockUntilConnected();
28 | leaderLatch = new LeaderLatch(zkClient, latchPath, id);
29 | leaderLatch.start();
30 | }
31 |
32 | public void blockUntilisLeader() throws Exception {
33 | leaderLatch.await();
34 | }
35 |
36 | public Participant currentLeader() throws Exception {
37 | return leaderLatch.getLeader();
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/benchmark/BenchmarkArgs.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.benchmark;
2 |
3 | import com.lexicalscope.jewel.cli.CommandLineInterface;
4 | import com.lexicalscope.jewel.cli.Option;
5 |
6 | /**
7 | * Created by xdong on 11/16/15.
8 | */
9 | @CommandLineInterface(application="kafka-spraynozzle-benchmark")
10 | public interface BenchmarkArgs {
11 | @Option(shortName="rt", defaultValue = "4", description="Reader thread count")
12 | int getReaderThreads();
13 | @Option(shortName="pt", defaultValue = "4", description="Post thread count")
14 | int getPostThreads();
15 | @Option(shortName="mps", defaultValue = "200000", description="Kafka Message Per Second, Per thread")
16 | int getMessagePerPerSecond();
17 | @Option(shortName="size", defaultValue = "2240", description="Kafka Message Size, in bytes")
18 | int getMessageSize();
19 | @Option(shortName="pc", defaultValue = "10", description="Post cost in micro-seconds (1/1000 of milliseconds)")
20 | int getPostCost();
21 | @Option(shortName="t", defaultValue = "90", description="Test time in seconds")
22 | int getTestTime();
23 | @Option(shortName="e", defaultValue = "1", description="enable real http test")
24 | int getEnableRealHttp();
25 | @Option(shortName="n", defaultValue = "4", description="http endpoint count")
26 | int getHttpEndpointCount();
27 | @Option(shortName="f", defaultValue = "0", description="Force round robin post instead of least-response time")
28 | int getForceRoundRobin();
29 | @Option(shortName="s", defaultValue = "20", description="How many messages to do in one batch")
30 | int getPackingPerPost();
31 |
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/benchmark/BenchmarkSpraynozzle.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.benchmark;
2 |
3 | import com.codahale.metrics.MetricRegistry;
4 | import com.github.tomakehurst.wiremock.WireMockServer;
5 | import com.github.tomakehurst.wiremock.http.*;
6 |
7 | import static com.github.tomakehurst.wiremock.http.RequestMethod.POST;
8 |
9 | import com.github.tomakehurst.wiremock.matching.RequestPattern;
10 | import com.github.tomakehurst.wiremock.stubbing.StubMapping;
11 | import com.lexicalscope.jewel.cli.ArgumentValidationException;
12 | import com.lexicalscope.jewel.cli.CliFactory;
13 | import com.uber.kafkaSpraynozzle.KafkaNoopFilter;
14 | import com.uber.kafkaSpraynozzle.KafkaPoster;
15 | import com.uber.kafkaSpraynozzle.KafkaReader;
16 | import org.apache.http.entity.ByteArrayEntity;
17 | import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
18 |
19 | import java.util.ArrayList;
20 | import java.util.Date;
21 | import java.util.List;
22 | import java.util.Objects;
23 | import java.util.concurrent.ConcurrentLinkedQueue;
24 | import java.util.concurrent.ExecutorService;
25 | import java.util.concurrent.Executors;
26 | import java.util.concurrent.TimeUnit;
27 | import java.util.concurrent.atomic.AtomicBoolean;
28 |
29 | /**
30 | * Created by xdong on 11/16/15.
31 | * The benchmark tries to pump messages into the queue, and then either takes out the queue items directly,
32 | * or starts a real local http server to test the KafkaPoster thread.
33 | *
34 | * By this benchmark, we can get the throughput of enqueue, get the load balancing tested.
35 | *
36 | */
37 | public class BenchmarkSpraynozzle {
38 |
39 |
40 | public static void main(String[] args) throws Exception {
41 | BenchmarkArgs benchmarkArgs;
42 | try {
43 | benchmarkArgs = CliFactory.parseArguments(BenchmarkArgs.class, args);
44 | } catch(ArgumentValidationException e) {
45 | System.err.println(e.getMessage());
46 | System.err.println(CliFactory.createCli(BenchmarkArgs.class).getHelpMessage());
47 | return;
48 | }
49 |
50 | System.out.print("\nTesting setting: \n");
51 | if (benchmarkArgs.getEnableRealHttp() == 1){
52 | System.out.print("Posting to endpoint, with endpoint[0] has 200ms delays.\n");
53 | }else{
54 | System.out.print("Skipping endpoint write.\n");
55 | }
56 | if (benchmarkArgs.getForceRoundRobin() == 1){
57 | System.out.print("Forcing round-robin.\n");
58 | }else{
59 | System.out.print("Least-response-time balancing.\n");
60 | }
61 | System.out.print(String.format("Packing %d items in post.\n", benchmarkArgs.getPackingPerPost()));
62 | System.out.println("\n");
63 |
64 | final long NANO_PER_SECOND = 1000L*1000L*1000L;
65 |
66 | ExecutorService executor = Executors.newFixedThreadPool(benchmarkArgs.getReaderThreads() + benchmarkArgs.getPostThreads() + 1);
67 | final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue();
68 | final ConcurrentLinkedQueue logQueue = new ConcurrentLinkedQueue();
69 | AtomicBoolean abortFlag = new AtomicBoolean(false);
70 | for (int i = 0; i < benchmarkArgs.getReaderThreads(); ++i){
71 | KafkaReader reader = new KafkaReader(new MetricRegistry(), queue, null);
72 | KafkaReaderDriver driver = new KafkaReaderDriver(reader, benchmarkArgs.getTestTime() * benchmarkArgs.getMessagePerPerSecond(),
73 | benchmarkArgs.getMessageSize(), benchmarkArgs.getMessagePerPerSecond(), abortFlag);
74 | executor.submit(driver);
75 | }
76 |
77 | final long[] serverCalledCount = new long[benchmarkArgs.getHttpEndpointCount()];
78 | long startingTime = new Date().getTime();
79 | List servers = new ArrayList(benchmarkArgs.getHttpEndpointCount());
80 | if (benchmarkArgs.getEnableRealHttp() == 1){
81 | // Http Connection Pooling stuff
82 | List urls = new ArrayList(benchmarkArgs.getHttpEndpointCount());
83 | for (int i = 0; i < benchmarkArgs.getHttpEndpointCount(); ++i){
84 | urls.add("http://localhost:" + (18982 + i) + "/endpoint");
85 | }
86 |
87 | for (int i = 0; i < benchmarkArgs.getHttpEndpointCount(); ++i){
88 | servers.add(new WireMockServer(18982 + i));
89 | final int serverIdx = i;
90 | servers.get(i).addStubMapping(new StubMapping(new RequestPattern(POST, "/endpoint"), ResponseDefinition.ok()));
91 | servers.get(i).addMockServiceRequestListener(
92 | new RequestListener() {
93 | @Override
94 | public void requestReceived(Request request, Response response) {
95 | serverCalledCount[serverIdx]++;
96 | }
97 | }
98 | );
99 | servers.get(i).start();
100 | if (i == 0){ /// introduce imbalanced server response.
101 | servers.get(0).addRequestProcessingDelay(200);
102 | }
103 | }
104 | final PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager();
105 | cm.setMaxTotal(benchmarkArgs.getPostThreads());
106 | cm.setDefaultMaxPerRoute(benchmarkArgs.getPostThreads());
107 |
108 | for (int i = 0; i < benchmarkArgs.getPostThreads(); ++i) {
109 | KafkaPoster poster = new KafkaPoster(new MetricRegistry(), queue, cm, urls, new KafkaNoopFilter(),
110 | 2000, 1000, 20, benchmarkArgs.getForceRoundRobin() == 1, true);
111 | executor.submit(poster);
112 | }
113 |
114 | }else {
115 | for (int i = 0; i < benchmarkArgs.getPostThreads(); ++i) {
116 | KafkaPosterSimulator poster = new KafkaPosterSimulator(benchmarkArgs.getPostCost(), queue, abortFlag);
117 | executor.submit(poster);
118 | }
119 | }
120 | long startNanoTime = System.nanoTime();
121 | while (System.nanoTime() - startNanoTime < (benchmarkArgs.getTestTime() + 20)* NANO_PER_SECOND ){
122 | Thread.sleep(1000);
123 | if (benchmarkArgs.getEnableRealHttp() == 1) {
124 | long currentTime = new Date().getTime();
125 | if (currentTime - startingTime > 10 * 1000){
126 | for (int i = 0; i < serverCalledCount.length; ++i){
127 | System.out.println(String.format("Server %d, called %d times", i, serverCalledCount[i]));
128 | }
129 | startingTime = currentTime;
130 | }
131 | }
132 | }
133 | abortFlag.set(true);
134 | executor.shutdown();
135 | executor.awaitTermination(300, TimeUnit.SECONDS);
136 | for (WireMockServer server: servers){
137 | server.shutdown();
138 | }
139 | }
140 | }
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/benchmark/KafkaPosterSimulator.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.benchmark;
2 |
3 | import org.apache.http.entity.ByteArrayEntity;
4 |
5 | import java.util.concurrent.ConcurrentLinkedQueue;
6 | import java.util.concurrent.atomic.AtomicBoolean;
7 |
8 | /**
9 | * This simulator is just taking out queue items, and throws them away.
10 | * If needed it can simulate the delay in post.
11 | * Created by xdong on 11/16/15.
12 | */
13 | public class KafkaPosterSimulator implements Runnable {
14 | public KafkaPosterSimulator(int postCostInMicroseconds, ConcurrentLinkedQueue queue,
15 | AtomicBoolean abortFlag){
16 | this.postCostInMicroseconds = postCostInMicroseconds;
17 | this.queue = queue;
18 | this.abort = abortFlag;
19 | }
20 | private AtomicBoolean abort = new AtomicBoolean(false);
21 | private long postCostInMicroseconds = 1L;
22 | private ConcurrentLinkedQueue queue;
23 | private final long NANO_PER_SECOND = 1000L*1000L*1000L;
24 | private final long PRINT_INTERVAL_IN_SEC = 5; /// every 5 seconds print one stats
25 |
26 | private static void sleepNanoSeconds(long nanoSecondsSleep){
27 | long start = System.nanoTime();
28 | long end;
29 | do{
30 | end = System.nanoTime();
31 | Thread.yield();
32 | }while(start + nanoSecondsSleep >= end);
33 | }
34 |
35 | public void run() {
36 | long startNanoTime = System.nanoTime();
37 | long simulatedPostCount = 0L;
38 | long sleepInNano = 1000;
39 | long lastReportNanoTime = startNanoTime;
40 | long threadId = Thread.currentThread().getId();
41 |
42 | while(!abort.get()){
43 | ByteArrayEntity entity = queue.poll();
44 | long currentNanoTime = System.nanoTime();
45 | long overallNanoTime = (currentNanoTime - startNanoTime + 1);
46 | if (entity != null) {
47 | simulatedPostCount += 1;
48 | long simulatedCostInMilli = overallNanoTime / 1000 / simulatedPostCount;
49 | while (simulatedCostInMilli < postCostInMicroseconds){
50 | sleepNanoSeconds(sleepInNano);
51 | currentNanoTime = System.nanoTime();
52 | overallNanoTime = (currentNanoTime - startNanoTime + 1);
53 | simulatedCostInMilli = overallNanoTime / 1000 / simulatedPostCount;
54 | }
55 | }
56 | if (currentNanoTime - lastReportNanoTime > PRINT_INTERVAL_IN_SEC * NANO_PER_SECOND){
57 | lastReportNanoTime = currentNanoTime;
58 | System.out.println(String.format("Post thread %d: Posting at %d message/sec.", threadId,
59 | simulatedPostCount * NANO_PER_SECOND/overallNanoTime));
60 |
61 | }
62 | }
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/benchmark/KafkaReaderDriver.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.benchmark;
2 |
3 | import com.uber.kafkaSpraynozzle.KafkaReader;
4 |
5 | import java.nio.ByteBuffer;
6 | import java.util.concurrent.atomic.AtomicBoolean;
7 |
8 | /**
9 | * This class takes in one KafkaReader, and test the code path where it enqueues
10 | * It is driven by a thread.
11 | */
12 | class KafkaReaderDriver implements Runnable {
13 | public KafkaReaderDriver(KafkaReader readerToTest, long packageCount,
14 | int packageSize, int packagePerSecond, AtomicBoolean abortFlag){
15 | reader = readerToTest;
16 | sleepNanoInterval = 1000000000L/packagePerSecond;
17 | this.packageCount = packageCount;
18 | this.packageSize = packageSize;
19 | this.packagePerSecond = packagePerSecond;
20 | abort = abortFlag;
21 | }
22 | private KafkaReader reader;
23 | private long sleepNanoInterval = 10000; // 10 microseconds, 0.01 milli
24 | private long packageCount = 1000000;
25 | private int packageSize = 2000; // 2KB
26 | private int packagePerSecond = 1000;
27 | private final long NANO_PER_SECOND = 1000L*1000L*1000L;
28 | private final long PRINT_INTERVAL_IN_SEC = 5; /// every 5 seconds print one stats
29 | private AtomicBoolean abort = new AtomicBoolean(false);
30 |
31 | private static void sleepNanoSeconds(long nanoSecondsSleep){
32 | long start = System.nanoTime();
33 | long end;
34 | do{
35 | end = System.nanoTime();
36 | Thread.yield();
37 | }while(start + nanoSecondsSleep >= end);
38 | }
39 |
40 | public void run() {
41 |
42 | long threadId = Thread.currentThread().getId();
43 | System.out.println("Starting reader thread " + threadId);
44 | int pushCount = 0;
45 | byte[] bytes = new byte[packageSize];
46 | ByteBuffer buffer = ByteBuffer.wrap(bytes);
47 | long overallStartTime = System.nanoTime();
48 | long lastReportTime = overallStartTime;
49 | long totalSleepingNanoSeconds = 0L;
50 | for(long i = 0L; i < packageCount && !abort.get(); ++i) {
51 | ByteBuffer message = buffer;
52 | Integer messageLen = buffer.array().length;
53 | pushCount = reader.enqueueData(pushCount, messageLen, message);
54 |
55 | long currentEndTime = System.nanoTime();
56 | long totalElapsedTime = (currentEndTime - overallStartTime + 1); /// avoid division by zero
57 | long currentMessagePerSecond = (i + 1) * NANO_PER_SECOND / totalElapsedTime ;
58 | /// target our packagePerSecond goal here
59 | if ( currentMessagePerSecond > packagePerSecond ) {
60 | sleepNanoSeconds(sleepNanoInterval);
61 | totalSleepingNanoSeconds += sleepNanoInterval;
62 | }
63 |
64 | if (currentEndTime - lastReportTime > PRINT_INTERVAL_IN_SEC * NANO_PER_SECOND){
65 | System.out.println(String.format("Reader Thread %d: Sustained %d message/sec, %d KiB/sec. %d percent idle", threadId,
66 | currentMessagePerSecond, currentMessagePerSecond * packageSize / 1000,
67 | totalSleepingNanoSeconds * 100 / totalElapsedTime ));
68 | lastReportTime = currentEndTime;
69 | }
70 | }
71 | }
72 |
73 | }
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/stats/NoopReporterFactory.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.stats;
2 |
3 | import com.codahale.metrics.MetricRegistry;
4 | import com.codahale.metrics.ScheduledReporter;
5 | import com.fasterxml.jackson.annotation.JsonTypeName;
6 |
7 | import java.io.IOException;
8 |
9 | @JsonTypeName("noop")
10 | public class NoopReporterFactory implements StatsReporterFactory {
11 | public NoopReporterFactory() {
12 | }
13 |
14 | @Override
15 | public ScheduledReporter build(MetricRegistry registry) {
16 | return null;
17 | }
18 |
19 | @Override
20 | public String toString() {
21 | return "NoopReporterFactory";
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/stats/StatsReporterFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2015 Uber Technologies, Inc.
3 | *
4 | * Permission is hereby granted, free of charge, to any person obtaining a copy
5 | * of this software and associated documentation files (the "Software"), to deal
6 | * in the Software without restriction, including without limitation the rights
7 | * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
8 | * copies of the Software, and to permit persons to whom the Software is
9 | * furnished to do so, subject to the following conditions:
10 | *
11 | * The above copyright notice and this permission notice shall be included in
12 | * all copies or substantial portions of the Software.
13 | *
14 | * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
15 | * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
16 | * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
17 | * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
18 | * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
19 | * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
20 | * THE SOFTWARE.
21 | */
22 | package com.uber.kafkaSpraynozzle.stats;
23 |
24 | import com.codahale.metrics.MetricRegistry;
25 | import com.codahale.metrics.ScheduledReporter;
26 |
27 | public interface StatsReporterFactory {
28 | ScheduledReporter build(MetricRegistry metricRegistry);
29 | }
30 |
--------------------------------------------------------------------------------
/src/main/java/com/uber/kafkaSpraynozzle/stats/StatsdReporterFactory.java:
--------------------------------------------------------------------------------
1 | package com.uber.kafkaSpraynozzle.stats;
2 |
3 | import com.codahale.metrics.MetricRegistry;
4 | import com.codahale.metrics.ScheduledReporter;
5 | import com.fasterxml.jackson.annotation.JsonTypeName;
6 | import com.fasterxml.jackson.core.type.TypeReference;
7 | import com.fasterxml.jackson.databind.ObjectMapper;
8 | import com.readytalk.metrics.StatsDReporter;
9 |
10 | import java.io.IOException;
11 | import java.util.HashMap;
12 | import java.util.Map;
13 |
14 | @JsonTypeName("statsd")
15 | public class StatsdReporterFactory implements StatsReporterFactory {
16 | private final String statsPrefix;
17 | private final String statsdHost;
18 | private final Integer statsdPort;
19 |
20 | public StatsdReporterFactory(String jsonConfig) throws IOException {
21 | // convert jsonConfig into Map
22 | Map configMap;
23 | try {
24 | configMap = new ObjectMapper().readValue(jsonConfig, new TypeReference>() {});
25 | } catch (IOException e) {
26 | throw new IllegalArgumentException("input is not json: " + jsonConfig);
27 | }
28 |
29 | this.statsPrefix = (String)configMap.get("statsPrefix");
30 | this.statsdHost = (String)configMap.get("host");
31 | this.statsdPort = (Integer)configMap.get("port");
32 | System.out.println("Connecting to statsd at: " + this.statsdHost + ":" + this.statsdPort);
33 | }
34 |
35 | @Override
36 | public ScheduledReporter build(MetricRegistry registry) {
37 | return StatsDReporter.forRegistry(registry)
38 | .prefixedWith(statsPrefix)
39 | .build(statsdHost, statsdPort);
40 | }
41 |
42 | @Override
43 | public String toString() {
44 | return "StatsdReporterFactory{" +
45 | "stats prefix=" + this.statsPrefix +
46 | ", statsd host=" + this.statsdHost +
47 | ", statsd port=" + this.statsdPort +
48 | "}";
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/src/main/node/test-server.js:
--------------------------------------------------------------------------------
1 | var cluster = require('cluster');
2 | var http = require('http');
3 |
4 | if (cluster.isMaster) {
5 | for(var i = 0; i < 8; i++) {
6 | cluster.fork();
7 | }
8 | } else {
9 | http.createServer(function(req, res) {
10 | console.log('Got a request!');
11 | req.on('data', function(chunk) { console.log(chunk.toString()); });
12 | req.on('end', res.end.bind(res, '{}'));
13 | }).listen(11235);
14 | }
15 |
--------------------------------------------------------------------------------
/src/test/java/com/uber/kafkaSpraynozzle/ListSerializerTest.java:
--------------------------------------------------------------------------------
1 |
2 | package com.uber.kafkaSpraynozzle;
3 |
4 | import kafka.common.InvalidMessageSizeException;
5 | import org.apache.http.entity.ByteArrayEntity;
6 | import org.junit.Test;
7 |
8 |
9 | import java.io.ByteArrayOutputStream;
10 | import java.io.IOException;
11 | import java.util.ArrayList;
12 | import java.util.List;
13 |
14 | /**
15 | * Created by xdong on 11/17/15.
16 | */
17 | public class ListSerializerTest {
18 |
19 | @Test
20 | public void SimpleCaseShouldWork(){
21 | int arrayLength = 10;
22 | List entities = new ArrayList(arrayLength);
23 | for (int i = 0; i < arrayLength; ++i){
24 | int elementLength = i * 10;
25 | ByteArrayOutputStream helper = new ByteArrayOutputStream();
26 | for (int j = 0; j < elementLength; ++j){
27 | helper.write(i * 1000 + j);
28 | }
29 | ByteArrayEntity entity = new ByteArrayEntity(helper.toByteArray());
30 | entities.add(entity);
31 | }
32 | ByteArrayEntity serialized = null;
33 | try {
34 | serialized = ListSerializer.toByteArrayEntity(entities);
35 | }catch(InvalidMessageSizeException ex){
36 | assert("corrupted message size".equals(ex.getMessage()));
37 | }catch(IOException ex){
38 | assert("IO exception during exception".equals(ex.getMessage()));
39 | }
40 | assert(serialized != null && serialized.getContentLength() > 100L);
41 |
42 | byte[] payload = ListSerializer.getByteArrayEntityBytes(serialized);
43 |
44 | List deserialized = null;
45 | try {
46 | deserialized = ListSerializer.toListOfByteArrayEntity(payload);
47 | }catch(IOException ex){
48 | assert("IO exception during exception".equals(ex.getMessage()));
49 | }
50 | assert(deserialized != null && deserialized.size() == entities.size());
51 | for (int i = 0; i < entities.size(); ++i){
52 | assert(entities.get(i).getContentLength() == deserialized.get(i).getContentLength());
53 | byte[] originalBytes = ListSerializer.getByteArrayEntityBytes(entities.get(i));
54 | byte[] pipedBytes = ListSerializer.getByteArrayEntityBytes(deserialized.get(i));
55 | assert(originalBytes.length == pipedBytes.length);
56 | for (int j = 0; j < originalBytes.length; ++j){
57 | assert(originalBytes[j] == pipedBytes[j]);
58 | }
59 | }
60 | }
61 | @Test
62 | public void ExceptionForCorruptedData(){
63 | int arrayLength = 10;
64 | List entities = new ArrayList(arrayLength);
65 | for (int i = 0; i < arrayLength; ++i){
66 | int elementLength = i * 10;
67 | ByteArrayOutputStream helper = new ByteArrayOutputStream();
68 | for (int j = 0; j < elementLength; ++j){
69 | helper.write(i * 1000 + j);
70 | }
71 | ByteArrayEntity entity = new ByteArrayEntity(helper.toByteArray());
72 | entities.add(entity);
73 | }
74 | ByteArrayEntity serialized = null;
75 | try {
76 | serialized = ListSerializer.toByteArrayEntity(entities);
77 | }catch(InvalidMessageSizeException ex){
78 | assert("corrupted message size".equals(ex.getMessage()));
79 | }catch(IOException ex){
80 | assert("IO exception during exception".equals(ex.getMessage()));
81 | }
82 | assert(serialized != null && serialized.getContentLength() > 100L);
83 |
84 | byte[] payload = ListSerializer.getByteArrayEntityBytes(serialized);
85 | for (int i = 0; i < payload.length; ++i){
86 | payload[i] += i;
87 | }
88 | boolean exceptionThrown = false;
89 | List deserialized = null;
90 | try {
91 | deserialized = ListSerializer.toListOfByteArrayEntity(payload);
92 | }catch(IOException ex){
93 | exceptionThrown = true;
94 |
95 | }catch(InvalidMessageSizeException ex){
96 | exceptionThrown = true;
97 | }
98 | assert(exceptionThrown);
99 | assert(deserialized == null);
100 | }
101 | }
102 |
--------------------------------------------------------------------------------