forked from apache/kafka
/
consumer_test.py
473 lines (376 loc) · 23.7 KB
/
consumer_test.py
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
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
from ducktape.mark import matrix
from ducktape.utils.util import wait_until
from ducktape.mark.resource import cluster
from kafkatest.tests.verifiable_consumer_test import VerifiableConsumerTest
from kafkatest.services.kafka import TopicPartition, quorum
import signal
class OffsetValidationTest(VerifiableConsumerTest):
TOPIC = "test_topic"
NUM_PARTITIONS = 1
def __init__(self, test_context):
super(OffsetValidationTest, self).__init__(test_context, num_consumers=3, num_producers=1,
num_zk=1, num_brokers=2, topics={
self.TOPIC : { 'partitions': self.NUM_PARTITIONS, 'replication-factor': 2 }
})
def rolling_bounce_consumers(self, consumer, keep_alive=0, num_bounces=5, clean_shutdown=True):
for _ in range(num_bounces):
for node in consumer.nodes[keep_alive:]:
consumer.stop_node(node, clean_shutdown)
wait_until(lambda: len(consumer.dead_nodes()) == 1,
timeout_sec=self.session_timeout_sec+5,
err_msg="Timed out waiting for the consumer to shutdown")
consumer.start_node(node)
self.await_all_members(consumer)
self.await_consumed_messages(consumer)
def bounce_all_consumers(self, consumer, keep_alive=0, num_bounces=5, clean_shutdown=True):
for _ in range(num_bounces):
for node in consumer.nodes[keep_alive:]:
consumer.stop_node(node, clean_shutdown)
wait_until(lambda: len(consumer.dead_nodes()) == self.num_consumers - keep_alive, timeout_sec=10,
err_msg="Timed out waiting for the consumers to shutdown")
for node in consumer.nodes[keep_alive:]:
consumer.start_node(node)
self.await_all_members(consumer)
self.await_consumed_messages(consumer)
def rolling_bounce_brokers(self, consumer, num_bounces=5, clean_shutdown=True):
for _ in range(num_bounces):
for node in self.kafka.nodes:
self.kafka.restart_node(node, clean_shutdown=True)
self.await_all_members(consumer)
self.await_consumed_messages(consumer)
def setup_consumer(self, topic, **kwargs):
# collect verifiable consumer events since this makes debugging much easier
consumer = super(OffsetValidationTest, self).setup_consumer(topic, **kwargs)
self.mark_for_collect(consumer, 'verifiable_consumer_stdout')
return consumer
@cluster(num_nodes=7)
@matrix(metadata_quorum=quorum.all_non_upgrade)
def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk):
"""
Verify correct consumer behavior when the brokers are consecutively restarted.
Setup: single Kafka cluster with one producer writing messages to a single topic with one
partition, an a set of consumers in the same group reading from the same topic.
- Start a producer which continues producing new messages throughout the test.
- Start up the consumers and wait until they've joined the group.
- In a loop, restart each broker consecutively, waiting for the group to stabilize between
each broker restart.
- Verify delivery semantics according to the failure type and that the broker bounces
did not cause unexpected group rebalances.
"""
partition = TopicPartition(self.TOPIC, 0)
producer = self.setup_producer(self.TOPIC)
# The consumers' session timeouts must exceed the time it takes for a broker to roll. Consumers are likely
# to see cluster metadata consisting of just a single alive broker in the case where the cluster has just 2
# brokers and the cluster is rolling (which is what is happening here). When the consumer sees a single alive
# broker, and then that broker rolls, the consumer will be unable to connect to the cluster until that broker
# completes its roll. In the meantime, the consumer group will move to the group coordinator on the other
# broker, and that coordinator will fail the consumer and trigger a group rebalance if its session times out.
# This test is asserting that no rebalances occur, so we increase the session timeout for this to be the case.
self.session_timeout_sec = 30
consumer = self.setup_consumer(self.TOPIC)
producer.start()
self.await_produced_messages(producer)
consumer.start()
self.await_all_members(consumer)
num_rebalances = consumer.num_rebalances()
# TODO: make this test work with hard shutdowns, which probably requires
# pausing before the node is restarted to ensure that any ephemeral
# nodes have time to expire
self.rolling_bounce_brokers(consumer, clean_shutdown=True)
unexpected_rebalances = consumer.num_rebalances() - num_rebalances
assert unexpected_rebalances == 0, \
"Broker rolling bounce caused %d unexpected group rebalances" % unexpected_rebalances
consumer.stop_all()
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
@cluster(num_nodes=7)
@matrix(clean_shutdown=[True], bounce_mode=["all", "rolling"], metadata_quorum=quorum.all_non_upgrade)
def test_consumer_bounce(self, clean_shutdown, bounce_mode, metadata_quorum=quorum.zk):
"""
Verify correct consumer behavior when the consumers in the group are consecutively restarted.
Setup: single Kafka cluster with one producer and a set of consumers in one group.
- Start a producer which continues producing new messages throughout the test.
- Start up the consumers and wait until they've joined the group.
- In a loop, restart each consumer, waiting for each one to rejoin the group before
restarting the rest.
- Verify delivery semantics according to the failure type.
"""
partition = TopicPartition(self.TOPIC, 0)
producer = self.setup_producer(self.TOPIC)
consumer = self.setup_consumer(self.TOPIC)
producer.start()
self.await_produced_messages(producer)
consumer.start()
self.await_all_members(consumer)
if bounce_mode == "all":
self.bounce_all_consumers(consumer, clean_shutdown=clean_shutdown)
else:
self.rolling_bounce_consumers(consumer, clean_shutdown=clean_shutdown)
consumer.stop_all()
if clean_shutdown:
# if the total records consumed matches the current position, we haven't seen any duplicates
# this can only be guaranteed with a clean shutdown
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
else:
# we may have duplicates in a hard failure
assert consumer.current_position(partition) <= consumer.total_consumed(), \
"Current position %d greater than the total number of consumed records %d" % \
(consumer.current_position(partition), consumer.total_consumed())
@cluster(num_nodes=7)
@matrix(clean_shutdown=[True], static_membership=[True, False], bounce_mode=["all", "rolling"], num_bounces=[5], metadata_quorum=quorum.all_non_upgrade)
def test_static_consumer_bounce(self, clean_shutdown, static_membership, bounce_mode, num_bounces, metadata_quorum=quorum.zk):
"""
Verify correct static consumer behavior when the consumers in the group are restarted. In order to make
sure the behavior of static members are different from dynamic ones, we take both static and dynamic
membership into this test suite.
Setup: single Kafka cluster with one producer and a set of consumers in one group.
- Start a producer which continues producing new messages throughout the test.
- Start up the consumers as static/dynamic members and wait until they've joined the group.
- In a loop, restart each consumer except the first member (note: may not be the leader), and expect no rebalance triggered
during this process if the group is in static membership.
"""
partition = TopicPartition(self.TOPIC, 0)
producer = self.setup_producer(self.TOPIC)
producer.start()
self.await_produced_messages(producer)
self.session_timeout_sec = 60
consumer = self.setup_consumer(self.TOPIC, static_membership=static_membership)
consumer.start()
self.await_all_members(consumer)
num_revokes_before_bounce = consumer.num_revokes_for_alive()
num_keep_alive = 1
if bounce_mode == "all":
self.bounce_all_consumers(consumer, keep_alive=num_keep_alive, num_bounces=num_bounces)
else:
self.rolling_bounce_consumers(consumer, keep_alive=num_keep_alive, num_bounces=num_bounces)
num_revokes_after_bounce = consumer.num_revokes_for_alive() - num_revokes_before_bounce
check_condition = num_revokes_after_bounce != 0
# under static membership, the live consumer shall not revoke any current running partitions,
# since there is no global rebalance being triggered.
if static_membership:
check_condition = num_revokes_after_bounce == 0
assert check_condition, \
"Total revoked count %d does not match the expectation of having 0 revokes as %d" % \
(num_revokes_after_bounce, check_condition)
consumer.stop_all()
if clean_shutdown:
# if the total records consumed matches the current position, we haven't seen any duplicates
# this can only be guaranteed with a clean shutdown
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
else:
# we may have duplicates in a hard failure
assert consumer.current_position(partition) <= consumer.total_consumed(), \
"Current position %d greater than the total number of consumed records %d" % \
(consumer.current_position(partition), consumer.total_consumed())
@cluster(num_nodes=7)
@matrix(bounce_mode=["all", "rolling"], metadata_quorum=quorum.all_non_upgrade)
def test_static_consumer_persisted_after_rejoin(self, bounce_mode, metadata_quorum=quorum.zk):
"""
Verify that the updated member.id(updated_member_id) caused by static member rejoin would be persisted. If not,
after the brokers rolling bounce, the migrated group coordinator would load the stale persisted member.id and
fence subsequent static member rejoin with updated_member_id.
- Start a producer which continues producing new messages throughout the test.
- Start up a static consumer and wait until it's up
- Restart the consumer and wait until it up, its member.id is supposed to be updated and persisted.
- Rolling bounce all the brokers and verify that the static consumer can still join the group and consumer messages.
"""
producer = self.setup_producer(self.TOPIC)
producer.start()
self.await_produced_messages(producer)
self.session_timeout_sec = 60
consumer = self.setup_consumer(self.TOPIC, static_membership=True)
consumer.start()
self.await_all_members(consumer)
# bounce the static member to trigger its member.id updated
if bounce_mode == "all":
self.bounce_all_consumers(consumer, num_bounces=1)
else:
self.rolling_bounce_consumers(consumer, num_bounces=1)
# rolling bounce all the brokers to trigger the group coordinator migration and verify updated member.id is persisted
# and reloaded successfully
self.rolling_bounce_brokers(consumer, num_bounces=1)
@cluster(num_nodes=10)
@matrix(num_conflict_consumers=[1, 2], fencing_stage=["stable", "all"], metadata_quorum=quorum.all_non_upgrade)
def test_fencing_static_consumer(self, num_conflict_consumers, fencing_stage, metadata_quorum=quorum.zk):
"""
Verify correct static consumer behavior when there are conflicting consumers with same group.instance.id.
- Start a producer which continues producing new messages throughout the test.
- Start up the consumers as static members and wait until they've joined the group. Some conflict consumers will be configured with
- the same group.instance.id.
- Let normal consumers and fencing consumers start at the same time, and expect only unique consumers left.
"""
partition = TopicPartition(self.TOPIC, 0)
producer = self.setup_producer(self.TOPIC)
producer.start()
self.await_produced_messages(producer)
self.session_timeout_sec = 60
consumer = self.setup_consumer(self.TOPIC, static_membership=True)
self.num_consumers = num_conflict_consumers
conflict_consumer = self.setup_consumer(self.TOPIC, static_membership=True)
# wait original set of consumer to stable stage before starting conflict members.
if fencing_stage == "stable":
consumer.start()
self.await_members(consumer, len(consumer.nodes))
conflict_consumer.start()
self.await_members(conflict_consumer, num_conflict_consumers)
self.await_members(consumer, len(consumer.nodes) - num_conflict_consumers)
wait_until(lambda: len(consumer.dead_nodes()) == num_conflict_consumers,
timeout_sec=10,
err_msg="Timed out waiting for the fenced consumers to stop")
else:
consumer.start()
conflict_consumer.start()
wait_until(lambda: len(consumer.joined_nodes()) + len(conflict_consumer.joined_nodes()) == len(consumer.nodes),
timeout_sec=self.session_timeout_sec,
err_msg="Timed out waiting for consumers to join, expected total %d joined, but only see %d joined from"
"normal consumer group and %d from conflict consumer group" % \
(len(consumer.nodes), len(consumer.joined_nodes()), len(conflict_consumer.joined_nodes()))
)
wait_until(lambda: len(consumer.dead_nodes()) + len(conflict_consumer.dead_nodes()) == len(conflict_consumer.nodes),
timeout_sec=self.session_timeout_sec,
err_msg="Timed out waiting for fenced consumers to die, expected total %d dead, but only see %d dead in"
"normal consumer group and %d dead in conflict consumer group" % \
(len(conflict_consumer.nodes), len(consumer.dead_nodes()), len(conflict_consumer.dead_nodes()))
)
@cluster(num_nodes=7)
@matrix(clean_shutdown=[True], enable_autocommit=[True, False], metadata_quorum=quorum.all_non_upgrade)
def test_consumer_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk):
partition = TopicPartition(self.TOPIC, 0)
consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit)
producer = self.setup_producer(self.TOPIC)
consumer.start()
self.await_all_members(consumer)
partition_owner = consumer.owner(partition)
assert partition_owner is not None
# startup the producer and ensure that some records have been written
producer.start()
self.await_produced_messages(producer)
# stop the partition owner and await its shutdown
consumer.kill_node(partition_owner, clean_shutdown=clean_shutdown)
wait_until(lambda: len(consumer.joined_nodes()) == (self.num_consumers - 1) and consumer.owner(partition) is not None,
timeout_sec=self.session_timeout_sec*2+5,
err_msg="Timed out waiting for consumer to close")
# ensure that the remaining consumer does some work after rebalancing
self.await_consumed_messages(consumer, min_messages=1000)
consumer.stop_all()
if clean_shutdown:
# if the total records consumed matches the current position, we haven't seen any duplicates
# this can only be guaranteed with a clean shutdown
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
else:
# we may have duplicates in a hard failure
assert consumer.current_position(partition) <= consumer.total_consumed(), \
"Current position %d greater than the total number of consumed records %d" % \
(consumer.current_position(partition), consumer.total_consumed())
# if autocommit is not turned on, we can also verify the last committed offset
if not enable_autocommit:
assert consumer.last_commit(partition) == consumer.current_position(partition), \
"Last committed offset %d did not match last consumed position %d" % \
(consumer.last_commit(partition), consumer.current_position(partition))
@cluster(num_nodes=7)
@matrix(clean_shutdown=[True, False], enable_autocommit=[True, False], metadata_quorum=quorum.all_non_upgrade)
def test_broker_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk):
partition = TopicPartition(self.TOPIC, 0)
consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit)
producer = self.setup_producer(self.TOPIC)
producer.start()
consumer.start()
self.await_all_members(consumer)
num_rebalances = consumer.num_rebalances()
# shutdown one of the brokers
# TODO: we need a way to target the coordinator instead of picking arbitrarily
self.kafka.signal_node(self.kafka.nodes[0], signal.SIGTERM if clean_shutdown else signal.SIGKILL)
# ensure that the consumers do some work after the broker failure
self.await_consumed_messages(consumer, min_messages=1000)
# verify that there were no rebalances on failover
assert num_rebalances == consumer.num_rebalances(), "Broker failure should not cause a rebalance"
consumer.stop_all()
# if the total records consumed matches the current position, we haven't seen any duplicates
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
# if autocommit is not turned on, we can also verify the last committed offset
if not enable_autocommit:
assert consumer.last_commit(partition) == consumer.current_position(partition), \
"Last committed offset %d did not match last consumed position %d" % \
(consumer.last_commit(partition), consumer.current_position(partition))
@cluster(num_nodes=7)
@matrix(metadata_quorum=quorum.all_non_upgrade)
def test_group_consumption(self, metadata_quorum=quorum.zk):
"""
Verifies correct group rebalance behavior as consumers are started and stopped.
In particular, this test verifies that the partition is readable after every
expected rebalance.
Setup: single Kafka cluster with a group of consumers reading from one topic
with one partition while the verifiable producer writes to it.
- Start the consumers one by one, verifying consumption after each rebalance
- Shutdown the consumers one by one, verifying consumption after each rebalance
"""
consumer = self.setup_consumer(self.TOPIC)
producer = self.setup_producer(self.TOPIC)
partition = TopicPartition(self.TOPIC, 0)
producer.start()
for num_started, node in enumerate(consumer.nodes, 1):
consumer.start_node(node)
self.await_members(consumer, num_started)
self.await_consumed_messages(consumer)
for num_stopped, node in enumerate(consumer.nodes, 1):
consumer.stop_node(node)
if num_stopped < self.num_consumers:
self.await_members(consumer, self.num_consumers - num_stopped)
self.await_consumed_messages(consumer)
assert consumer.current_position(partition) == consumer.total_consumed(), \
"Total consumed records %d did not match consumed position %d" % \
(consumer.total_consumed(), consumer.current_position(partition))
assert consumer.last_commit(partition) == consumer.current_position(partition), \
"Last committed offset %d did not match last consumed position %d" % \
(consumer.last_commit(partition), consumer.current_position(partition))
class AssignmentValidationTest(VerifiableConsumerTest):
TOPIC = "test_topic"
NUM_PARTITIONS = 6
def __init__(self, test_context):
super(AssignmentValidationTest, self).__init__(test_context, num_consumers=3, num_producers=0,
num_zk=1, num_brokers=2, topics={
self.TOPIC : { 'partitions': self.NUM_PARTITIONS, 'replication-factor': 1 },
})
@cluster(num_nodes=6)
@matrix(assignment_strategy=["org.apache.kafka.clients.consumer.RangeAssignor",
"org.apache.kafka.clients.consumer.RoundRobinAssignor",
"org.apache.kafka.clients.consumer.StickyAssignor"], metadata_quorum=quorum.all_non_upgrade)
def test_valid_assignment(self, assignment_strategy, metadata_quorum=quorum.zk):
"""
Verify assignment strategy correctness: each partition is assigned to exactly
one consumer instance.
Setup: single Kafka cluster with a set of consumers in the same group.
- Start the consumers one by one
- Validate assignment after every expected rebalance
"""
consumer = self.setup_consumer(self.TOPIC, assignment_strategy=assignment_strategy)
for num_started, node in enumerate(consumer.nodes, 1):
consumer.start_node(node)
self.await_members(consumer, num_started)
assert self.valid_assignment(self.TOPIC, self.NUM_PARTITIONS, consumer.current_assignment()), \
"expected valid assignments of %d partitions when num_started %d: %s" % \
(self.NUM_PARTITIONS, num_started, \
[(str(node.account), a) for node, a in consumer.current_assignment().items()])