class StreamsBrokerDownResilience(Test): """ This test validates that Streams is resilient to a broker being down longer than specified timeouts in configs """ inputTopic = "streamsResilienceSource" outputTopic = "streamsResilienceSink" num_messages = 5 def __init__(self, test_context): super(StreamsBrokerDownResilience, self).__init__(test_context=test_context) self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk, topics={ self.inputTopic: { 'partitions': 3, 'replication-factor': 1 }, self.outputTopic: { 'partitions': 1, 'replication-factor': 1 } }) def get_consumer(self, num_messages): return VerifiableConsumer(self.test_context, 1, self.kafka, self.outputTopic, "stream-broker-resilience-verify-consumer", max_messages=num_messages) def get_producer(self, num_messages): return VerifiableProducer(self.test_context, 1, self.kafka, self.inputTopic, max_messages=num_messages, acks=1) def assert_produce_consume(self, test_state, num_messages=5): producer = self.get_producer(num_messages) producer.start() wait_until(lambda: producer.num_acked >= num_messages, timeout_sec=30, err_msg="At %s failed to send messages " % test_state) consumer = self.get_consumer(num_messages) consumer.start() wait_until( lambda: consumer.total_consumed() >= num_messages, timeout_sec=60, err_msg="At %s streams did not process messages in 60 seconds " % test_state) @staticmethod def get_configs(extra_configs=""): # Consumer max.poll.interval > min(max.block.ms, ((retries + 1) * request.timeout) consumer_poll_ms = "consumer.max.poll.interval.ms=50000" retries_config = "producer.retries=2" request_timeout = "producer.request.timeout.ms=15000" max_block_ms = "producer.max.block.ms=30000" # java code expects configs in key=value,key=value format updated_configs = consumer_poll_ms + "," + retries_config + "," + request_timeout + "," + max_block_ms + extra_configs return updated_configs def wait_for_verification(self, processor, message, file, num_lines=1): wait_until(lambda: self.verify_from_file(processor, message, file ) >= num_lines, timeout_sec=60, err_msg="Did expect to read '%s' from %s" % (message, processor.node.account)) @staticmethod def verify_from_file(processor, message, file): result = processor.node.account.ssh_output("grep '%s' %s | wc -l" % (message, file), allow_fail=False) return int(result) def setUp(self): self.zk.start() def test_streams_resilient_to_broker_down(self): self.kafka.start() # Broker should be down over 2x of retries * timeout ms # So with (2 * 15000) = 30 seconds, we'll set downtime to 70 seconds broker_down_time_in_seconds = 70 processor = StreamsBrokerDownResilienceService(self.test_context, self.kafka, self.get_configs()) processor.start() # until KIP-91 is merged we'll only send 5 messages to assert Kafka Streams is running before taking the broker down # After KIP-91 is merged we'll continue to send messages the duration of the test self.assert_produce_consume("before_broker_stop") node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) time.sleep(broker_down_time_in_seconds) self.kafka.start_node(node) self.assert_produce_consume("after_broker_stop") self.kafka.stop() def test_streams_runs_with_broker_down_initially(self): self.kafka.start() node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) configs = self.get_configs( extra_configs=",application.id=starting_wo_broker_id") # start streams with broker down initially processor = StreamsBrokerDownResilienceService(self.test_context, self.kafka, configs) processor.start() processor_2 = StreamsBrokerDownResilienceService( self.test_context, self.kafka, configs) processor_2.start() processor_3 = StreamsBrokerDownResilienceService( self.test_context, self.kafka, configs) processor_3.start() broker_unavailable_message = "Broker may not be available" # verify streams instances unable to connect to broker, kept trying self.wait_for_verification(processor, broker_unavailable_message, processor.LOG_FILE, 100) self.wait_for_verification(processor_2, broker_unavailable_message, processor_2.LOG_FILE, 100) self.wait_for_verification(processor_3, broker_unavailable_message, processor_3.LOG_FILE, 100) # now start broker self.kafka.start_node(node) # assert streams can process when starting with broker down self.assert_produce_consume("running_with_broker_down_initially", num_messages=9) message = "processed3messages" # need to show all 3 instances processed messages self.wait_for_verification(processor, message, processor.STDOUT_FILE) self.wait_for_verification(processor_2, message, processor_2.STDOUT_FILE) self.wait_for_verification(processor_3, message, processor_3.STDOUT_FILE) self.kafka.stop() def test_streams_should_scale_in_while_brokers_down(self): self.kafka.start() configs = self.get_configs( extra_configs=",application.id=shutdown_with_broker_down") processor = StreamsBrokerDownResilienceService(self.test_context, self.kafka, configs) processor.start() processor_2 = StreamsBrokerDownResilienceService( self.test_context, self.kafka, configs) processor_2.start() processor_3 = StreamsBrokerDownResilienceService( self.test_context, self.kafka, configs) processor_3.start() # need to wait for rebalance once self.wait_for_verification( processor_3, "State transition from REBALANCING to RUNNING", processor_3.LOG_FILE) # assert streams can process when starting with broker down self.assert_produce_consume("waiting for rebalance to complete", num_messages=9) message = "processed3messages" self.wait_for_verification(processor, message, processor.STDOUT_FILE) self.wait_for_verification(processor_2, message, processor_2.STDOUT_FILE) self.wait_for_verification(processor_3, message, processor_3.STDOUT_FILE) node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) processor.stop() processor_2.stop() shutdown_message = "Complete shutdown of streams resilience test app now" self.wait_for_verification(processor, shutdown_message, processor.STDOUT_FILE) self.wait_for_verification(processor_2, shutdown_message, processor_2.STDOUT_FILE) self.kafka.start_node(node) self.assert_produce_consume( "sending_message_after_stopping_streams_instance_bouncing_broker", num_messages=9) self.wait_for_verification(processor_3, "processed9messages", processor_3.STDOUT_FILE) self.kafka.stop()
class LogDirFailureTest(ProduceConsumeValidateTest): """ Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose ordering guarantees. Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, we might exit early if some messages are duplicated (though not an issue here since producer retries==0) Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively consumed messages. Since we run the producer to completion before running the consumer, this is a reliable indicator that nothing is left to consume. """ def __init__(self, test_context): """:type test_context: ducktape.tests.test.TestContext""" super(LogDirFailureTest, self).__init__(test_context=test_context) self.topic1 = "test_topic_1" self.topic2 = "test_topic_2" self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService( test_context, num_nodes=3, zk=self.zk, topics={ self.topic1: { "partitions": 1, "replication-factor": 3, "configs": { "min.insync.replicas": 2 } }, self.topic2: { "partitions": 1, "replication-factor": 3, "configs": { "min.insync.replicas": 1 } } }, # Set log.roll.ms to 3 seconds so that broker will detect disk error sooner when it creates log segment # Otherwise broker will still be able to read/write the log file even if the log directory is inaccessible. server_prop_overides=[ [config_property.LOG_FLUSH_INTERVAL_MESSAGE, "5"], [ config_property. REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS, "60000" ], [config_property.LOG_ROLL_TIME_MS, "3000"] ]) self.producer_throughput = 1000 self.num_producers = 1 self.num_consumers = 1 def setUp(self): self.zk.start() def min_cluster_size(self): """Override this since we're adding services outside of the constructor""" return super(LogDirFailureTest, self).min_cluster_size( ) + self.num_producers * 2 + self.num_consumers * 2 @cluster(num_nodes=9) @matrix(bounce_broker=[False, True], broker_type=["leader", "follower"], security_protocol=["PLAINTEXT"]) def test_replication_with_disk_failure(self, bounce_broker, security_protocol, broker_type): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by brokers is still available for consumption in the face of various failure scenarios. Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2 and another topic with partitions=3, replication-factor=3, and min.insync.replicas=1 - Produce messages in the background - Consume messages in the background - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9) - When done driving failures, stop producing, and finish consuming - Validate that every acked message was consumed """ self.kafka.security_protocol = security_protocol self.kafka.interbroker_security_protocol = security_protocol self.kafka.start() try: # Initialize producer/consumer for topic1 self.producer = VerifiableProducer( self.test_context, self.num_producers, self.kafka, self.topic1, throughput=self.producer_throughput) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-1", new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) self.start_producer_and_consumer() # Get a replica of the partition of topic1 and make its first log directory offline by changing the log dir's permission. # We assume that partition of topic1 is created in the first log directory of respective brokers. broker_node = select_node(self, broker_type, self.topic1) broker_idx = self.kafka.idx(broker_node) assert broker_idx in self.kafka.isr_idx_list(self.topic1), \ "Broker %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1))) self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_1)) cmd = "chmod a-w %s -R" % (KafkaService.DATA_LOG_DIR_1) broker_node.account.ssh(cmd, allow_fail=False) if bounce_broker: self.kafka.restart_node(broker_node, clean_shutdown=True) # Verify the following: # 1) The broker with offline log directory is not the leader of the partition of topic1 # 2) The broker with offline log directory is not in the ISR # 3) The broker with offline log directory is still online # 4) Messages can still be produced and consumed from topic1 wait_until( lambda: self.kafka.leader(self.topic1, partition=0 ) != broker_node, timeout_sec=60, err_msg= "Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic1)) assert self.kafka.alive( broker_node), "Broker %d should be still online" % (broker_idx) wait_until( lambda: broker_idx not in self.kafka.isr_idx_list(self.topic1), timeout_sec=60, err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1)))) self.stop_producer_and_consumer() self.validate() # Shutdown all other brokers so that the broker with offline log dir is the only online broker offline_nodes = [] for node in self.kafka.nodes: if broker_node != node: offline_nodes.append(node) self.logger.debug("Hard shutdown broker %d" % (self.kafka.idx(node))) self.kafka.stop_node(node) # Verify the following: # 1) The broker with offline directory is the only in-sync broker of the partition of topic2 # 2) Messages can still be produced and consumed from topic2 self.producer = VerifiableProducer( self.test_context, self.num_producers, self.kafka, self.topic2, throughput=self.producer_throughput, offline_nodes=offline_nodes) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-2", new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) self.start_producer_and_consumer() assert self.kafka.isr_idx_list(self.topic2) == [broker_idx], \ "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic2, str([broker_idx])) self.stop_producer_and_consumer() self.validate() except BaseException as e: for s in self.test_context.services: self.mark_for_collect(s) raise
class LogDirFailureTest(ProduceConsumeValidateTest): """ Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose ordering guarantees. Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, we might exit early if some messages are duplicated (though not an issue here since producer retries==0) Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively consumed messages. Since we run the producer to completion before running the consumer, this is a reliable indicator that nothing is left to consume. """ def __init__(self, test_context): """:type test_context: ducktape.tests.test.TestContext""" super(LogDirFailureTest, self).__init__(test_context=test_context) self.topic1 = "test_topic_1" self.topic2 = "test_topic_2" self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={ self.topic1: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 1}}, self.topic2: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 2}} }, # Set log.roll.ms to 3 seconds so that broker will detect disk error sooner when it creates log segment # Otherwise broker will still be able to read/write the log file even if the log directory is inaccessible. server_prop_overides=[ [config_property.OFFSETS_TOPIC_NUM_PARTITIONS, "1"], [config_property.LOG_FLUSH_INTERVAL_MESSAGE, "5"], [config_property.REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS, "60000"], [config_property.LOG_ROLL_TIME_MS, "3000"] ]) self.producer_throughput = 1000 self.num_producers = 1 self.num_consumers = 1 def setUp(self): self.zk.start() def min_cluster_size(self): """Override this since we're adding services outside of the constructor""" return super(LogDirFailureTest, self).min_cluster_size() + self.num_producers * 2 + self.num_consumers * 2 @cluster(num_nodes=9) @matrix(bounce_broker=[False, True], broker_type=["leader", "follower"], security_protocol=["PLAINTEXT"]) def test_replication_with_disk_failure(self, bounce_broker, security_protocol, broker_type): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by brokers is still available for consumption in the face of various failure scenarios. Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2 and another topic with partitions=3, replication-factor=3, and min.insync.replicas=1 - Produce messages in the background - Consume messages in the background - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9) - When done driving failures, stop producing, and finish consuming - Validate that every acked message was consumed """ self.kafka.security_protocol = security_protocol self.kafka.interbroker_security_protocol = security_protocol self.kafka.start() try: # Initialize producer/consumer for topic2 self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic2, throughput=self.producer_throughput) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-1", consumer_timeout_ms=60000, message_validator=is_int) self.start_producer_and_consumer() # Get a replica of the partition of topic2 and make its log directory offline by changing the log dir's permission. # We assume that partition of topic2 is created in the second log directory of respective brokers. broker_node = select_node(self, broker_type, self.topic2) broker_idx = self.kafka.idx(broker_node) assert broker_idx in self.kafka.isr_idx_list(self.topic2), \ "Broker %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic2))) # Verify that topic1 and the consumer offset topic is in the first log directory and topic2 is in the second log directory topic_1_partition_0 = KafkaService.DATA_LOG_DIR_1 + "/test_topic_1-0" topic_2_partition_0 = KafkaService.DATA_LOG_DIR_2 + "/test_topic_2-0" offset_topic_partition_0 = KafkaService.DATA_LOG_DIR_1 + "/__consumer_offsets-0" for path in [topic_1_partition_0, topic_2_partition_0, offset_topic_partition_0]: assert path_exists(broker_node, path), "%s should exist" % path self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_2)) cmd = "chmod a-w %s -R" % (KafkaService.DATA_LOG_DIR_2) broker_node.account.ssh(cmd, allow_fail=False) if bounce_broker: self.kafka.restart_node(broker_node, clean_shutdown=True) # Verify the following: # 1) The broker with offline log directory is not the leader of the partition of topic2 # 2) The broker with offline log directory is not in the ISR # 3) The broker with offline log directory is still online # 4) Messages can still be produced and consumed from topic2 wait_until(lambda: self.kafka.leader(self.topic2, partition=0) != broker_node, timeout_sec=60, err_msg="Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic2)) assert self.kafka.alive(broker_node), "Broker %d should be still online" % (broker_idx) wait_until(lambda: broker_idx not in self.kafka.isr_idx_list(self.topic2), timeout_sec=60, err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic2)))) self.stop_producer_and_consumer() self.validate() # Shutdown all other brokers so that the broker with offline log dir is the only online broker offline_nodes = [] for node in self.kafka.nodes: if broker_node != node: offline_nodes.append(node) self.logger.debug("Hard shutdown broker %d" % (self.kafka.idx(node))) self.kafka.stop_node(node) # Verify the following: # 1) The broker with offline directory is the only in-sync broker of the partition of topic1 # 2) Messages can still be produced and consumed from topic1 self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic1, throughput=self.producer_throughput, offline_nodes=offline_nodes) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-2", consumer_timeout_ms=90000, message_validator=is_int) self.consumer_start_timeout_sec = 90 self.start_producer_and_consumer() assert self.kafka.isr_idx_list(self.topic1) == [broker_idx], \ "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic1, str([broker_idx])) self.stop_producer_and_consumer() self.validate() except BaseException as e: for s in self.test_context.services: self.mark_for_collect(s) raise
class FetchFromFollowerTest(ProduceConsumeValidateTest): RACK_AWARE_REPLICA_SELECTOR = "org.apache.kafka.common.replica.RackAwareReplicaSelector" METADATA_MAX_AGE_MS = 3000 def __init__(self, test_context): super(FetchFromFollowerTest, self).__init__(test_context=test_context) self.jmx_tool = JmxTool(test_context, jmx_poll_ms=100) self.topic = "test_topic" self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={ self.topic: { "partitions": 1, "replication-factor": 3, "configs": { "min.insync.replicas": 1 } }, }, server_prop_overides=[[ "replica.selector.class", self.RACK_AWARE_REPLICA_SELECTOR ]], per_node_server_prop_overrides={ 1: [("broker.rack", "rack-a")], 2: [("broker.rack", "rack-b")], 3: [("broker.rack", "rack-c")] }) self.producer_throughput = 1000 self.num_producers = 1 self.num_consumers = 1 def min_cluster_size(self): return super(FetchFromFollowerTest, self).min_cluster_size( ) + self.num_producers * 2 + self.num_consumers * 2 def setUp(self): self.zk.start() self.kafka.start() @cluster(num_nodes=9) def test_consumer_preferred_read_replica(self): """ This test starts up brokers with "broker.rack" and "replica.selector.class" configurations set. The replica selector is set to the rack-aware implementation. One of the brokers has a different rack than the other two. We then use a console consumer with the "client.rack" set to the same value as the differing broker. After producing some records, we verify that the client has been informed of the preferred replica and that all the records are properly consumed. """ # Find the leader, configure consumer to be on a different rack leader_node = self.kafka.leader(self.topic, 0) leader_idx = self.kafka.idx(leader_node) non_leader_idx = 2 if leader_idx != 2 else 1 non_leader_rack = "rack-b" if leader_idx != 2 else "rack-a" self.logger.debug("Leader %d %s" % (leader_idx, leader_node)) self.logger.debug("Non-Leader %d %s" % (non_leader_idx, non_leader_rack)) self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, client_id="console-consumer", group_id="test-consumer-group-1", consumer_timeout_ms=60000, message_validator=is_int, consumer_properties={ "client.rack": non_leader_rack, "metadata.max.age.ms": self.METADATA_MAX_AGE_MS }) # Start up and let some data get produced self.start_producer_and_consumer() time.sleep(self.METADATA_MAX_AGE_MS * 2. / 1000) consumer_node = self.consumer.nodes[0] consumer_idx = self.consumer.idx(consumer_node) read_replica_attribute = "preferred-read-replica" read_replica_mbean = "kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s,topic=%s,partition=%d" % \ ("console-consumer", self.topic, 0) self.jmx_tool.jmx_object_names = [read_replica_mbean] self.jmx_tool.jmx_attributes = [read_replica_attribute] self.jmx_tool.start_jmx_tool(consumer_idx, consumer_node) # Wait for at least one interval of "metadata.max.age.ms" time.sleep(self.METADATA_MAX_AGE_MS * 2. / 1000) # Read the JMX output self.jmx_tool.read_jmx_output(consumer_idx, consumer_node) all_captured_preferred_read_replicas = defaultdict(int) self.logger.debug(self.jmx_tool.jmx_stats) for ts, data in self.jmx_tool.jmx_stats[0].items(): for k, v in data.items(): if k.endswith(read_replica_attribute): all_captured_preferred_read_replicas[int(v)] += 1 self.logger.debug("Saw the following preferred read replicas %s", dict(all_captured_preferred_read_replicas.items())) assert all_captured_preferred_read_replicas[non_leader_idx] > 0, \ "Expected to see broker %d (%s) as a preferred replica" % (non_leader_idx, non_leader_rack) # Validate consumed messages self.stop_producer_and_consumer() self.validate()
class StreamsBrokerDownResilience(Test): """ This test validates that Streams is resilient to a broker being down longer than specified timeouts in configs """ inputTopic = "streamsResilienceSource" outputTopic = "streamsResilienceSink" num_messages = 5 def __init__(self, test_context): super(StreamsBrokerDownResilience, self).__init__(test_context=test_context) self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk, topics={ self.inputTopic: { 'partitions': 1, 'replication-factor': 1 }, self.outputTopic: { 'partitions': 1, 'replication-factor': 1 } }) def get_consumer(self): return VerifiableConsumer(self.test_context, 1, self.kafka, self.outputTopic, "stream-broker-resilience-verify-consumer", max_messages=self.num_messages) def get_producer(self): return VerifiableProducer(self.test_context, 1, self.kafka, self.inputTopic, max_messages=self.num_messages, acks=1) def assert_produce_consume(self, test_state): producer = self.get_producer() producer.start() wait_until(lambda: producer.num_acked > 0, timeout_sec=30, err_msg="At %s failed to send messages " % test_state) consumer = self.get_consumer() consumer.start() wait_until( lambda: consumer.total_consumed() > 0, timeout_sec=120, err_msg="At %s streams did not process messages in 120 seconds " % test_state) def setUp(self): self.zk.start() def test_streams_resilient_to_broker_down(self): self.kafka.start() # Consumer max.poll.interval > min(max.block.ms, ((retries + 1) * request.timeout) consumer_poll_ms = "consumer.max.poll.interval.ms=50000" retries_config = "producer.retries=2" request_timeout = "producer.request.timeout.ms=15000" max_block_ms = "producer.max.block.ms=30000" # Broker should be down over 2x of retries * timeout ms # So with (2 * 15000) = 30 seconds, we'll set downtime to 70 seconds broker_down_time_in_seconds = 70 # java code expects configs in key=value,key=value format updated_configs = consumer_poll_ms + "," + retries_config + "," + request_timeout + "," + max_block_ms processor = StreamsBrokerDownResilienceService(self.test_context, self.kafka, updated_configs) processor.start() # until KIP-91 is merged we'll only send 5 messages to assert Kafka Streams is running before taking the broker down # After KIP-91 is merged we'll continue to send messages the duration of the test self.assert_produce_consume("before_broker_stop") node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) time.sleep(broker_down_time_in_seconds) self.kafka.start_node(node) self.assert_produce_consume("after_broker_stop") self.kafka.stop()
class ReplicationTest(Test): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by brokers is still available for consumption in the face of various failure scenarios.""" def __init__(self, test_context): """:type test_context: ducktape.tests.test.TestContext""" super(ReplicationTest, self).__init__(test_context=test_context) self.topic = "test_topic" self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={ self.topic: { "partitions": 3, "replication-factor": 3, "min.insync.replicas": 2 } }) self.producer_throughput = 10000 self.num_producers = 1 self.num_consumers = 1 def setUp(self): self.zk.start() self.kafka.start() def min_cluster_size(self): """Override this since we're adding services outside of the constructor""" return super( ReplicationTest, self).min_cluster_size() + self.num_producers + self.num_consumers def run_with_failure(self, failure): """This is the top-level test template. The steps are: Produce messages in the background while driving some failure condition When done driving failures, immediately stop producing Consume all messages Validate that messages acked by brokers were consumed Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose ordering guarantees. Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, we might exit early if some messages are duplicated (though not an issue here since producer retries==0) Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively consumed messages. Since we run the producer to completion before running the consumer, this is a reliable indicator that nothing is left to consume. """ self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=3000) # Produce in a background thread while driving broker failures self.producer.start() if not wait_until(lambda: self.producer.num_acked > 5, timeout_sec=5): raise RuntimeError( "Producer failed to start in a reasonable amount of time.") failure() self.producer.stop() self.acked = self.producer.acked self.not_acked = self.producer.not_acked self.logger.info("num not acked: %d" % self.producer.num_not_acked) self.logger.info("num acked: %d" % self.producer.num_acked) # Consume all messages self.consumer.start() self.consumer.wait() self.consumed = self.consumer.messages_consumed[1] self.logger.info("num consumed: %d" % len(self.consumed)) # Check produced vs consumed success, msg = self.validate() if not success: self.mark_for_collect(self.producer) assert success, msg def clean_shutdown(self): """Discover leader node for our topic and shut it down cleanly.""" self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGTERM) def hard_shutdown(self): """Discover leader node for our topic and shut it down with a hard kill.""" self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGKILL) def clean_bounce(self): """Chase the leader of one partition and restart it cleanly.""" for i in range(5): prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=True) def hard_bounce(self): """Chase the leader and restart it cleanly.""" for i in range(5): prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=False) # Wait long enough for previous leader to probably be awake again time.sleep(6) def validate(self): """Check that produced messages were consumed.""" success = True msg = "" if len(set(self.consumed)) != len(self.consumed): # There are duplicates. This is ok, so report it but don't fail the test msg += "There are duplicate messages in the log\n" if not set(self.consumed).issuperset(set(self.acked)): # Every acked message must appear in the logs. I.e. consumed messages must be superset of acked messages. acked_minus_consumed = set(self.producer.acked) - set( self.consumed) success = False msg += "At least one acked message did not appear in the consumed messages. acked_minus_consumed: " + str( acked_minus_consumed) if not success: # Collect all the data logs if there was a failure self.mark_for_collect(self.kafka) return success, msg def test_clean_shutdown(self): self.run_with_failure(self.clean_shutdown) def test_hard_shutdown(self): self.run_with_failure(self.hard_shutdown) def test_clean_bounce(self): self.run_with_failure(self.clean_bounce) def test_hard_bounce(self): self.run_with_failure(self.hard_bounce)