1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
|
import glob
import logging
import os
import select
import shlex
import shutil
import socket
import subprocess
import tempfile
from threading import Thread, Event
import time
import unittest
from kafka.client08 import *
def get_open_port():
sock = socket.socket()
sock.bind(('',0))
port = sock.getsockname()[1]
sock.close()
return port
def build_kafka_classpath():
baseDir = "./kafka-src"
jars = []
jars += glob.glob(os.path.join(baseDir, "project/boot/scala-2.8.0/lib/*.jar"))
jars += glob.glob(os.path.join(baseDir, "core/target/scala_2.8.0/*.jar"))
jars += glob.glob(os.path.join(baseDir, "core/lib/*.jar"))
jars += glob.glob(os.path.join(baseDir, "core/lib_managed/scala_2.8.0/compile/*.jar"))
jars += glob.glob(os.path.join(baseDir, "core/target/scala-2.8.0/kafka_2.8.0-*.jar"))
jars += glob.glob(os.path.join(baseDir, "/Users/mumrah/.ivy2/cache/org.slf4j/slf4j-api/jars/slf4j-api-1.6.4.jar"))
cp = ":".join(["."] + [os.path.abspath(jar) for jar in jars])
cp += ":" + os.path.abspath(os.path.join(baseDir, "conf/log4j.properties"))
return cp
class KafkaFixture(Thread):
def __init__(self, host, port):
Thread.__init__(self)
self.port = port
self.capture = ""
self.shouldDie = Event()
self.tmpDir = tempfile.mkdtemp()
print("tmp dir: %s" % self.tmpDir)
def run(self):
# Create the log directory
logDir = os.path.join(self.tmpDir, 'logs')
os.mkdir(logDir)
stdout = open(os.path.join(logDir, 'stdout'), 'w')
# Create the config file
logConfig = "test/resources/log4j.properties"
configFile = os.path.join(self.tmpDir, 'server.properties')
f = open('test/resources/server.properties', 'r')
props = f.read()
f = open(configFile, 'w')
f.write(props % {'kafka.port': self.port, 'kafka.tmp.dir': logDir, 'kafka.partitions': 2})
f.close()
# Start Kafka
args = shlex.split("java -Xmx256M -server -Dlog4j.configuration=%s -cp %s kafka.Kafka %s" % (logConfig, build_kafka_classpath(), configFile))
proc = subprocess.Popen(args, bufsize=1, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, env={"JMX_PORT":"%d" % get_open_port()})
killed = False
while True:
(rlist, wlist, xlist) = select.select([proc.stdout], [], [], 1)
if proc.stdout in rlist:
read = proc.stdout.readline()
stdout.write(read)
stdout.flush()
self.capture += read
if self.shouldDie.is_set():
proc.terminate()
killed = True
if proc.poll() is not None:
#shutil.rmtree(self.tmpDir)
if killed:
break
else:
raise RuntimeError("Kafka died. Aborting.")
def wait_for(self, target, timeout=10):
t1 = time.time()
while True:
t2 = time.time()
if t2-t1 >= timeout:
return False
if target in self.capture:
return True
time.sleep(0.100)
def close(self):
self.shouldDie.set()
class ExternalKafkaFixture(object):
def __init__(self, host, port):
print("Using already running Kafka at %s:%d" % (host, port))
def close(self):
pass
class TestKafkaClient(unittest.TestCase):
@classmethod
def setUpClass(cls):
if os.environ.has_key('KAFKA_URI'):
parse = urlparse(os.environ['KAFKA_URI'])
(host, port) = (parse.hostname, parse.port)
cls.server = ExternalKafkaFixture(host, port)
cls.client = KafkaClient(host, port)
else:
port = get_open_port()
cls.server = KafkaFixture("localhost", port)
cls.server.start()
cls.server.wait_for("Kafka server started")
cls.client = KafkaClient("localhost", port)
@classmethod
def tearDownClass(cls):
cls.client.close()
cls.server.close()
#####################
# Produce Tests #
#####################
def test_produce_many_simple(self):
produce = ProduceRequest("test_produce_many_simple", 0, messages=[
KafkaProtocol.create_message("Test message %d" % i) for i in range(100)
])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 100)
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 100)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 200)
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 200)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 300)
def test_produce_10k_simple(self):
produce = ProduceRequest("test_produce_10k_simple", 0, messages=[
KafkaProtocol.create_message("Test message %d" % i) for i in range(10000)
])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_10k_simple", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 10000)
def test_produce_many_gzip(self):
message1 = KafkaProtocol.create_gzip_message(["Gzipped 1 %d" % i for i in range(100)])
message2 = KafkaProtocol.create_gzip_message(["Gzipped 2 %d" % i for i in range(100)])
produce = ProduceRequest("test_produce_many_gzip", 0, messages=[message1, message2])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_gzip", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 200)
def test_produce_many_snappy(self):
message1 = KafkaProtocol.create_snappy_message(["Snappy 1 %d" % i for i in range(100)])
message2 = KafkaProtocol.create_snappy_message(["Snappy 2 %d" % i for i in range(100)])
produce = ProduceRequest("test_produce_many_snappy", 0, messages=[message1, message2])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_snappy", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 200)
def test_produce_mixed(self):
message1 = KafkaProtocol.create_message("Just a plain message")
message2 = KafkaProtocol.create_gzip_message(["Gzipped %d" % i for i in range(100)])
message3 = KafkaProtocol.create_snappy_message(["Snappy %d" % i for i in range(100)])
produce = ProduceRequest("test_produce_mixed", 0, messages=[message1, message2, message3])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_mixed", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 201)
def test_produce_100k_gzipped(self):
produce = ProduceRequest("test_produce_100k_gzipped", 0, messages=[
KafkaProtocol.create_gzip_message(["Gzipped %d" % i for i in range(100000)])
])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
(offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_100k_gzipped", 0, -1, 1)])
self.assertEquals(offset.offsets[0], 100000)
#####################
# Consume Tests #
#####################
def test_consume_none(self):
fetch = FetchRequest("test_consume_none", 0, 0, 1024)
fetch_resp = self.client.send_fetch_request([fetch]).next()
self.assertEquals(fetch_resp.error, 0)
self.assertEquals(fetch_resp.topic, "test_consume_none")
self.assertEquals(fetch_resp.partition, 0)
messages = list(fetch_resp.messages)
self.assertEquals(len(messages), 0)
def test_produce_consume(self):
produce = ProduceRequest("test_produce_consume", 0, messages=[
KafkaProtocol.create_message("Just a test message"),
KafkaProtocol.create_message("Message with a key", "foo"),
])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
fetch = FetchRequest("test_produce_consume", 0, 0, 1024)
fetch_resp = self.client.send_fetch_request([fetch]).next()
self.assertEquals(fetch_resp.error, 0)
messages = list(fetch_resp.messages)
self.assertEquals(len(messages), 2)
self.assertEquals(messages[0].offset, 0)
self.assertEquals(messages[0].message.value, "Just a test message")
self.assertEquals(messages[0].message.key, None)
self.assertEquals(messages[1].offset, 1)
self.assertEquals(messages[1].message.value, "Message with a key")
self.assertEquals(messages[1].message.key, "foo")
def test_produce_consume_many(self):
produce = ProduceRequest("test_produce_consume_many", 0, messages=[
KafkaProtocol.create_message("Test message %d" % i) for i in range(100)
])
for resp in self.client.send_produce_request([produce]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
# 1024 is not enough for 100 messages...
fetch1 = FetchRequest("test_produce_consume_many", 0, 0, 1024)
(fetch_resp1,) = self.client.send_fetch_request([fetch1])
self.assertEquals(fetch_resp1.error, 0)
self.assertEquals(fetch_resp1.highwaterMark, 100)
messages = list(fetch_resp1.messages)
self.assertTrue(len(messages) < 100)
# 10240 should be enough
fetch2 = FetchRequest("test_produce_consume_many", 0, 0, 10240)
(fetch_resp2,) = self.client.send_fetch_request([fetch2])
self.assertEquals(fetch_resp2.error, 0)
self.assertEquals(fetch_resp2.highwaterMark, 100)
messages = list(fetch_resp2.messages)
self.assertEquals(len(messages), 100)
for i, message in enumerate(messages):
self.assertEquals(message.offset, i)
self.assertEquals(message.message.value, "Test message %d" % i)
self.assertEquals(message.message.key, None)
def test_produce_consume_two_partitions(self):
produce1 = ProduceRequest("test_produce_consume_two_partitions", 0, messages=[
KafkaProtocol.create_message("Partition 0 %d" % i) for i in range(10)
])
produce2 = ProduceRequest("test_produce_consume_two_partitions", 1, messages=[
KafkaProtocol.create_message("Partition 1 %d" % i) for i in range(10)
])
for resp in self.client.send_produce_request([produce1, produce2]):
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 0)
return
fetch1 = FetchRequest("test_produce_consume_two_partitions", 0, 0, 1024)
fetch2 = FetchRequest("test_produce_consume_two_partitions", 1, 0, 1024)
fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2])
self.assertEquals(fetch_resp1.error, 0)
self.assertEquals(fetch_resp1.highwaterMark, 10)
messages = list(fetch_resp1.messages)
self.assertEquals(len(messages), 10)
for i, message in enumerate(messages):
self.assertEquals(message.offset, i)
self.assertEquals(message.message.value, "Partition 0 %d" % i)
self.assertEquals(message.message.key, None)
self.assertEquals(fetch_resp2.error, 0)
self.assertEquals(fetch_resp2.highwaterMark, 10)
messages = list(fetch_resp2.messages)
self.assertEquals(len(messages), 10)
for i, message in enumerate(messages):
self.assertEquals(message.offset, i)
self.assertEquals(message.message.value, "Partition 1 %d" % i)
self.assertEquals(message.message.key, None)
####################
# Offset Tests #
####################
def test_commit_fetch_offsets(self):
req = OffsetCommitRequest("test_commit_fetch_offsets", 0, 42, "metadata")
(resp,) = self.client.send_offset_commit_request("group", [req])
self.assertEquals(resp.error, 0)
req = OffsetFetchRequest("test_commit_fetch_offsets", 0)
(resp,) = self.client.send_offset_fetch_request("group", [req])
self.assertEquals(resp.error, 0)
self.assertEquals(resp.offset, 42)
self.assertEquals(resp.metadata, "metadata")
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
unittest.main()
|