def move_start_offset(self): """We move the start offset of the topic by writing really old messages and waiting for them to be cleaned up. """ producer = VerifiableProducer(self.test_context, 1, self.kafka, self.topic, throughput=-1, enable_idempotence=True, create_time=1000) producer.start() wait_until(lambda: producer.num_acked > 0, timeout_sec=30, err_msg="Failed to get an acknowledgement for %ds" % 30) # Wait 8 seconds to let the topic be seeded with messages that will # be deleted. The 8 seconds is important, since we should get 2 deleted # segments in this period based on the configured log roll time and the # retention check interval. time.sleep(8) producer.stop() self.logger.info("Seeded topic with %d messages which will be deleted" %\ producer.num_acked) # Since the configured check interval is 5 seconds, we wait another # 6 seconds to ensure that at least one more cleaning so that the last # segment is deleted. An altenate to using timeouts is to poll each # partition until the log start offset matches the end offset. The # latter is more robust. time.sleep(6)
class TestBounce(Test): """Sanity checks on verifiable producer service class with cluster roll.""" def __init__(self, test_context): super(TestBounce, self).__init__(test_context) self.topic = "topic" self.zk = ZookeeperService(test_context, num_nodes=1) if quorum.for_test( test_context) == quorum.zk else None self.kafka = KafkaService( test_context, num_nodes=1, zk=self.zk, topics={self.topic: { "partitions": 1, "replication-factor": 1 }}, controller_num_nodes_override=3 if quorum.for_test(test_context) == quorum.remote_raft else 1) self.num_messages = 1000 def create_producer(self): # This will produce to source kafka cluster self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=self.topic, max_messages=self.num_messages, throughput=self.num_messages // 10) def setUp(self): if self.zk: self.zk.start() @cluster(num_nodes=6) @parametrize(metadata_quorum=quorum.remote_raft) @cluster(num_nodes=4) @parametrize(metadata_quorum=quorum.colocated_raft) @cluster(num_nodes=4) @parametrize(metadata_quorum=quorum.zk) def test_simple_run(self, metadata_quorum): """ Test that we can start VerifiableProducer on the current branch snapshot version, and verify that we can produce a small number of messages both before and after a subsequent roll. """ self.kafka.start() for first_time in [True, False]: self.create_producer() self.producer.start() wait_until( lambda: self.producer.num_acked > 5, timeout_sec=15, err_msg= "Producer failed to start in a reasonable amount of time.") self.producer.wait() num_produced = self.producer.num_acked assert num_produced == self.num_messages, "num_produced: %d, num_messages: %d" % ( num_produced, self.num_messages) if first_time: self.producer.stop() if self.kafka.quorum_info.using_raft and self.kafka.remote_controller_quorum: self.kafka.remote_controller_quorum.restart_cluster() self.kafka.restart_cluster()
class StreamsOptimizedTest(Test): """ Test doing upgrades of a Kafka Streams application that is un-optimized initially then optimized """ input_topic = 'inputTopic' aggregation_topic = 'aggregationTopic' reduce_topic = 'reduceTopic' join_topic = 'joinTopic' operation_pattern = 'AGGREGATED\|REDUCED\|JOINED' stopped_message = 'OPTIMIZE_TEST Streams Stopped' def __init__(self, test_context): super(StreamsOptimizedTest, self).__init__(test_context) self.topics = { self.input_topic: { 'partitions': 6 }, self.aggregation_topic: { 'partitions': 6 }, self.reduce_topic: { 'partitions': 6 }, self.join_topic: { 'partitions': 6 } } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.input_topic, throughput=1000, acks=1) def test_upgrade_optimized_topology(self): self.zookeeper.start() self.kafka.start() processor1 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processor2 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processor3 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processors = [processor1, processor2, processor3] # produce records continually during the test self.producer.start() # start all processors unoptimized for processor in processors: self.set_topics(processor) processor.CLEAN_NODE_ENABLED = False self.verify_running_repartition_topic_count(processor, 4) self.verify_processing(processors, verify_individual_operations=False) stop_processors(processors, self.stopped_message) # start again with topology optimized for processor in processors: processor.OPTIMIZED_CONFIG = 'all' self.verify_running_repartition_topic_count(processor, 1) self.verify_processing(processors, verify_individual_operations=True) stop_processors(processors, self.stopped_message) self.producer.stop() self.kafka.stop() self.zookeeper.stop() @staticmethod def verify_running_repartition_topic_count(processor, repartition_topic_count): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.start() monitor.wait_until( 'REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' % repartition_topic_count, timeout_sec=120, err_msg= "Never saw 'REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' message " % repartition_topic_count + str(processor.node.account)) def verify_processing(self, processors, verify_individual_operations): for processor in processors: if not self.all_source_subtopology_tasks(processor): if verify_individual_operations: for operation in self.operation_pattern.split('\|'): self.do_verify(processor, operation) else: self.do_verify(processor, self.operation_pattern) else: self.logger.info( "Skipping processor %s with all source tasks" % processor.node.account) def do_verify(self, processor, pattern): self.logger.info("Verifying %s processing pattern in STDOUT_FILE" % pattern) with processor.node.account.monitor_log( processor.STDOUT_FILE) as monitor: monitor.wait_until( pattern, timeout_sec=60, err_msg="Never saw processing of %s " % pattern + str(processor.node.account)) def all_source_subtopology_tasks(self, processor): retries = 0 while retries < 5: found = list( processor.node.account.ssh_capture( "sed -n 's/.*current active tasks: \[\(\(0_[0-9], \)\{3\}0_[0-9]\)\].*/\1/p' %s" % processor.LOG_FILE, allow_fail=True)) self.logger.info("Returned %s from assigned task check" % found) if len(found) > 0: return True retries += 1 time.sleep(1) return False def set_topics(self, processor): processor.INPUT_TOPIC = self.input_topic processor.AGGREGATION_TOPIC = self.aggregation_topic processor.REDUCE_TOPIC = self.reduce_topic processor.JOIN_TOPIC = self.join_topic
class StreamsCooperativeRebalanceUpgradeTest(Test): """ Test of a rolling upgrade from eager rebalance to cooperative rebalance """ source_topic = "source" sink_topic = "sink" task_delimiter = "#" report_interval = "1000" processing_message = "Processed [0-9]* records so far" stopped_message = "COOPERATIVE-REBALANCE-TEST-CLIENT-CLOSED" running_state_msg = "STREAMS in a RUNNING State" cooperative_turned_off_msg = "Eager rebalancing enabled now for upgrade from %s" cooperative_enabled_msg = "Cooperative rebalancing enabled now" first_bounce_phase = "first_bounce_phase-" second_bounce_phase = "second_bounce_phase-" # !!CAUTION!!: THIS LIST OF VERSIONS IS FIXED, NO VERSIONS MUST BE ADDED streams_eager_rebalance_upgrade_versions = [ str(LATEST_0_10_0), str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1), str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3) ] def __init__(self, test_context): super(StreamsCooperativeRebalanceUpgradeTest, self).__init__(test_context) self.topics = { self.source_topic: { 'partitions': 9 }, self.sink_topic: { 'partitions': 9 } } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.source_topic, throughput=1000, acks=1) @matrix(upgrade_from_version=streams_eager_rebalance_upgrade_versions) def test_upgrade_to_cooperative_rebalance(self, upgrade_from_version): self.zookeeper.start() self.kafka.start() processor1 = CooperativeRebalanceUpgradeService( self.test_context, self.kafka) processor2 = CooperativeRebalanceUpgradeService( self.test_context, self.kafka) processor3 = CooperativeRebalanceUpgradeService( self.test_context, self.kafka) processors = [processor1, processor2, processor3] # produce records continually during the test self.producer.start() # start all processors without upgrade_from config; normal operations mode self.logger.info("Starting all streams clients in normal running mode") for processor in processors: processor.set_version(upgrade_from_version) self.set_props(processor) processor.CLEAN_NODE_ENABLED = False # can't use state as older version don't have state listener # so just verify up and running verify_running(processor, self.processing_message) # all running rebalancing has ceased for processor in processors: self.verify_processing(processor, self.processing_message) # first rolling bounce with "upgrade.from" config set previous_phase = "" self.maybe_upgrade_rolling_bounce_and_verify(processors, previous_phase, self.first_bounce_phase, upgrade_from_version) # All nodes processing, rebalancing has ceased for processor in processors: self.verify_processing( processor, self.first_bounce_phase + self.processing_message) # second rolling bounce without "upgrade.from" config self.maybe_upgrade_rolling_bounce_and_verify(processors, self.first_bounce_phase, self.second_bounce_phase) # All nodes processing, rebalancing has ceased for processor in processors: self.verify_processing( processor, self.second_bounce_phase + self.processing_message) # now verify tasks are unique for processor in processors: self.get_tasks_for_processor(processor) self.logger.info("Active tasks %s" % processor.active_tasks) overlapping_tasks = processor1.active_tasks.intersection( processor2.active_tasks) assert len(overlapping_tasks) == int(0), \ "Final task assignments are not unique %s %s" % (processor1.active_tasks, processor2.active_tasks) overlapping_tasks = processor1.active_tasks.intersection( processor3.active_tasks) assert len(overlapping_tasks) == int(0), \ "Final task assignments are not unique %s %s" % (processor1.active_tasks, processor3.active_tasks) overlapping_tasks = processor2.active_tasks.intersection( processor3.active_tasks) assert len(overlapping_tasks) == int(0), \ "Final task assignments are not unique %s %s" % (processor2.active_tasks, processor3.active_tasks) # test done close all down stop_processors(processors, self.second_bounce_phase + self.stopped_message) self.producer.stop() self.kafka.stop() self.zookeeper.stop() def maybe_upgrade_rolling_bounce_and_verify(self, processors, previous_phase, current_phase, upgrade_from_version=None): for processor in processors: # stop the processor in prep for setting "update.from" or removing "update.from" verify_stopped(processor, previous_phase + self.stopped_message) # upgrade to version with cooperative rebalance processor.set_version("") processor.set_upgrade_phase(current_phase) if upgrade_from_version is not None: # need to remove minor version numbers for check of valid upgrade from numbers upgrade_version = upgrade_from_version[:upgrade_from_version. rfind('.')] rebalance_mode_msg = self.cooperative_turned_off_msg % upgrade_version else: upgrade_version = None rebalance_mode_msg = self.cooperative_enabled_msg self.set_props(processor, upgrade_version) node = processor.node with node.account.monitor_log( processor.STDOUT_FILE) as stdout_monitor: with node.account.monitor_log( processor.LOG_FILE) as log_monitor: processor.start() # verify correct rebalance mode either turned off for upgrade or enabled after upgrade log_monitor.wait_until( rebalance_mode_msg, timeout_sec=60, err_msg="Never saw '%s' message " % rebalance_mode_msg + str(processor.node.account)) # verify rebalanced into a running state rebalance_msg = current_phase + self.running_state_msg stdout_monitor.wait_until( rebalance_msg, timeout_sec=60, err_msg="Never saw '%s' message " % rebalance_msg + str(processor.node.account)) # verify processing verify_processing_msg = current_phase + self.processing_message stdout_monitor.wait_until( verify_processing_msg, timeout_sec=60, err_msg="Never saw '%s' message " % verify_processing_msg + str(processor.node.account)) def verify_processing(self, processor, pattern): self.logger.info("Verifying %s processing pattern in STDOUT_FILE" % pattern) with processor.node.account.monitor_log( processor.STDOUT_FILE) as monitor: monitor.wait_until( pattern, timeout_sec=60, err_msg="Never saw processing of %s " % pattern + str(processor.node.account)) def get_tasks_for_processor(self, processor): retries = 0 while retries < 5: found_tasks = list( processor.node.account.ssh_capture( "grep TASK-ASSIGNMENTS %s | tail -n 1" % processor.STDOUT_FILE, allow_fail=True)) self.logger.info("Returned %s from assigned task check" % found_tasks) if len(found_tasks) > 0: task_string = str(found_tasks[0]).strip() self.logger.info("Converted %s from assigned task check" % task_string) processor.set_tasks(task_string) return retries += 1 time.sleep(1) return def set_props(self, processor, upgrade_from=None): processor.SOURCE_TOPIC = self.source_topic processor.SINK_TOPIC = self.sink_topic processor.REPORT_INTERVAL = self.report_interval processor.UPGRADE_FROM = upgrade_from
class ReplicaVerificationToolTest(Test): """ Tests ReplicaVerificationTool """ def __init__(self, test_context): super(ReplicaVerificationToolTest, self).__init__(test_context) self.num_zk = 1 self.num_brokers = 2 self.messages_received_count = 0 self.topics = { TOPIC: {'partitions': 1, 'replication-factor': 2} } self.zk = ZookeeperService(test_context, self.num_zk) self.kafka = None self.producer = None self.replica_verifier = None def setUp(self): self.zk.start() def start_kafka(self, security_protocol, interbroker_security_protocol): self.kafka = KafkaService( self.test_context, self.num_brokers, self.zk, security_protocol=security_protocol, interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) self.kafka.start() def start_replica_verification_tool(self, security_protocol): self.replica_verifier = ReplicaVerificationTool(self.test_context, 1, self.kafka, TOPIC, report_interval_ms=REPORT_INTERVAL_MS, security_protocol=security_protocol) self.replica_verifier.start() def start_producer(self, max_messages, acks, timeout): # This will produce to kafka cluster self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, acks=acks, max_messages=max_messages) current_acked = self.producer.num_acked self.logger.info("current_acked = %s" % current_acked) self.producer.start() wait_until(lambda: acks == 0 or self.producer.num_acked >= current_acked + max_messages, timeout_sec=timeout, err_msg="Timeout awaiting messages to be produced and acked") def stop_producer(self): self.producer.stop() def test_replica_lags(self, security_protocol='PLAINTEXT'): """ Tests ReplicaVerificationTool :return: None """ self.start_kafka(security_protocol, security_protocol) self.start_replica_verification_tool(security_protocol) self.start_producer(max_messages=10, acks=-1, timeout=15) # Verify that there is no lag in replicas and is correctly reported by ReplicaVerificationTool wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) == 0, timeout_sec=10, err_msg="Timed out waiting to reach zero replica lags.") self.stop_producer() self.start_producer(max_messages=1000, acks=0, timeout=5) # Verify that there is lag in replicas and is correctly reported by ReplicaVerificationTool wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) > 0, timeout_sec=10, err_msg="Timed out waiting to reach non-zero number of replica lags.")
class EndToEndTest(Test): """This class provides a shared template for tests which follow the common pattern of: - produce to a topic in the background - consume from that topic in the background - run some logic, e.g. fail topic leader etc. - perform validation """ DEFAULT_TOPIC_CONFIG = {"partitions": 2, "replication-factor": 1} def __init__(self, test_context, topic="test_topic", topic_config=DEFAULT_TOPIC_CONFIG): super(EndToEndTest, self).__init__(test_context=test_context) self.topic = topic self.topic_config = topic_config self.records_consumed = [] self.last_consumed_offsets = {} def create_zookeeper(self, num_nodes=1, **kwargs): self.zk = ZookeeperService(self.test_context, num_nodes=num_nodes, **kwargs) def create_kafka(self, num_nodes=1, **kwargs): group_metadata_config = { "partitions": num_nodes, "replication-factor": min(num_nodes, 3), "configs": { "cleanup.policy": "compact" } } topics = { self.topic: self.topic_config, "__consumer_offsets": group_metadata_config } self.kafka = KafkaService(self.test_context, num_nodes=num_nodes, zk=self.zk, topics=topics, **kwargs) def create_consumer(self, num_nodes=1, group_id="test_group", **kwargs): self.consumer = VerifiableConsumer( self.test_context, num_nodes=num_nodes, kafka=self.kafka, topic=self.topic, group_id=group_id, on_record_consumed=self.on_record_consumed, **kwargs) def create_producer(self, num_nodes=1, throughput=1000, **kwargs): self.producer = VerifiableProducer(self.test_context, num_nodes=num_nodes, kafka=self.kafka, topic=self.topic, throughput=throughput, **kwargs) def on_record_consumed(self, record, node): partition = TopicPartition(record["topic"], record["partition"]) record_id = int(record["value"]) offset = record["offset"] self.last_consumed_offsets[partition] = offset self.records_consumed.append(record_id) def await_consumed_offsets(self, last_acked_offsets, timeout_sec): def has_finished_consuming(): for partition, offset in last_acked_offsets.iteritems(): if not partition in self.last_consumed_offsets: return False if self.last_consumed_offsets[partition] < offset: return False return True wait_until(has_finished_consuming, timeout_sec=timeout_sec, err_msg="Consumer failed to consume up to offsets %s after waiting %ds." %\ (str(last_acked_offsets), timeout_sec)) def _collect_all_logs(self): for s in self.test_context.services: self.mark_for_collect(s) def await_startup(self, min_records=5, timeout_sec=30): try: wait_until(lambda: self.consumer.total_consumed() >= min_records, timeout_sec=timeout_sec, err_msg="Timed out after %ds while awaiting initial record delivery of %d records" %\ (timeout_sec, min_records)) except BaseException: self._collect_all_logs() raise def run_validation(self, min_records=5000, producer_timeout_sec=30, consumer_timeout_sec=30, enable_idempotence=False): try: wait_until(lambda: self.producer.num_acked > min_records, timeout_sec=producer_timeout_sec, err_msg="Producer failed to produce messages for %ds." %\ producer_timeout_sec) self.logger.info("Stopping producer after writing up to offsets %s" %\ str(self.producer.last_acked_offsets)) self.producer.stop() self.await_consumed_offsets(self.producer.last_acked_offsets, consumer_timeout_sec) self.consumer.stop() self.validate(enable_idempotence) except BaseException: self._collect_all_logs() raise def validate(self, enable_idempotence): self.logger.info("Number of acked records: %d" % len(self.producer.acked)) self.logger.info("Number of consumed records: %d" % len(self.records_consumed)) def check_lost_data(missing_records): return self.kafka.search_data_files(self.topic, missing_records) succeeded, error_msg = validate_delivery(self.producer.acked, self.records_consumed, enable_idempotence, check_lost_data) # Collect all logs if validation fails if not succeeded: self._collect_all_logs() assert succeeded, error_msg
class EndToEndTest(Test): """This class provides a shared template for tests which follow the common pattern of: - produce to a topic in the background - consume from that topic in the background - run some logic, e.g. fail topic leader etc. - perform validation """ DEFAULT_TOPIC_CONFIG = {"partitions": 2, "replication-factor": 1} def __init__(self, test_context, topic="test_topic", topic_config=DEFAULT_TOPIC_CONFIG): super(EndToEndTest, self).__init__(test_context=test_context) self.topic = topic self.topic_config = topic_config self.records_consumed = [] self.last_consumed_offsets = {} def create_zookeeper(self, num_nodes=1, **kwargs): self.zk = ZookeeperService(self.test_context, num_nodes=num_nodes, **kwargs) def create_kafka(self, num_nodes=1, **kwargs): group_metadata_config = { "partitions": num_nodes, "replication-factor": min(num_nodes, 3), "configs": {"cleanup.policy": "compact"} } topics = { self.topic: self.topic_config, "__consumer_offsets": group_metadata_config } self.kafka = KafkaService(self.test_context, num_nodes=num_nodes, zk=self.zk, topics=topics, **kwargs) def create_consumer(self, num_nodes=1, group_id="test_group", **kwargs): self.consumer = VerifiableConsumer(self.test_context, num_nodes=num_nodes, kafka=self.kafka, topic=self.topic, group_id=group_id, on_record_consumed=self.on_record_consumed, **kwargs) def create_producer(self, num_nodes=1, throughput=1000, **kwargs): self.producer = VerifiableProducer(self.test_context, num_nodes=num_nodes, kafka=self.kafka, topic=self.topic, throughput=throughput, **kwargs) def on_record_consumed(self, record, node): partition = TopicPartition(record["topic"], record["partition"]) record_id = int(record["value"]) offset = record["offset"] self.last_consumed_offsets[partition] = offset self.records_consumed.append(record_id) def await_consumed_offsets(self, last_acked_offsets, timeout_sec): def has_finished_consuming(): for partition, offset in last_acked_offsets.iteritems(): if not partition in self.last_consumed_offsets: return False if self.last_consumed_offsets[partition] < offset: return False return True wait_until(has_finished_consuming, timeout_sec=timeout_sec, err_msg="Consumer failed to consume up to offsets %s after waiting %ds." %\ (str(last_acked_offsets), timeout_sec)) def _collect_all_logs(self): for s in self.test_context.services: self.mark_for_collect(s) def await_startup(self, min_records=5, timeout_sec=30): try: wait_until(lambda: self.consumer.total_consumed() >= min_records, timeout_sec=timeout_sec, err_msg="Timed out after %ds while awaiting initial record delivery of %d records" %\ (timeout_sec, min_records)) except BaseException: self._collect_all_logs() raise def run_validation(self, min_records=5000, producer_timeout_sec=30, consumer_timeout_sec=30, enable_idempotence=False): try: wait_until(lambda: self.producer.num_acked > min_records, timeout_sec=producer_timeout_sec, err_msg="Producer failed to produce messages for %ds." %\ producer_timeout_sec) self.logger.info("Stopping producer after writing up to offsets %s" %\ str(self.producer.last_acked_offsets)) self.producer.stop() self.await_consumed_offsets(self.producer.last_acked_offsets, consumer_timeout_sec) self.consumer.stop() self.validate(enable_idempotence) except BaseException: self._collect_all_logs() raise def validate(self, enable_idempotence): self.logger.info("Number of acked records: %d" % len(self.producer.acked)) self.logger.info("Number of consumed records: %d" % len(self.records_consumed)) def check_lost_data(missing_records): return self.kafka.search_data_files(self.topic, missing_records) succeeded, error_msg = validate_delivery(self.producer.acked, self.records_consumed, enable_idempotence, check_lost_data) # Collect all logs if validation fails if not succeeded: self._collect_all_logs() assert succeeded, error_msg
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)
class StreamsStaticMembershipTest(Test): """ Tests using static membership when broker points to minimum supported version (2.3) or higher. """ input_topic = 'inputTopic' pattern = 'PROCESSED' running_message = 'REBALANCING -> RUNNING' stopped_message = 'Static membership test closed' def __init__(self, test_context): super(StreamsStaticMembershipTest, self).__init__(test_context) self.topics = { self.input_topic: { 'partitions': 18 }, } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.input_topic, throughput=1000, acks=1) def test_rolling_bounces_will_not_trigger_rebalance_under_static_membership( self): self.zookeeper.start() self.kafka.start() numThreads = 3 processor1 = StaticMemberTestService(self.test_context, self.kafka, "consumer-A", numThreads) processor2 = StaticMemberTestService(self.test_context, self.kafka, "consumer-B", numThreads) processor3 = StaticMemberTestService(self.test_context, self.kafka, "consumer-C", numThreads) processors = [processor1, processor2, processor3] self.producer.start() for processor in processors: processor.CLEAN_NODE_ENABLED = False self.set_topics(processor) verify_running(processor, self.running_message) self.verify_processing(processors) # do several rolling bounces num_bounces = 3 for i in range(0, num_bounces): for processor in processors: verify_stopped(processor, self.stopped_message) verify_running(processor, self.running_message) stable_generation = -1 for processor in processors: generations = extract_generation_from_logs(processor) num_bounce_generations = num_bounces * numThreads assert num_bounce_generations <= len(generations), \ "Smaller than minimum expected %d generation messages, actual %d" % (num_bounce_generations, len(generations)) for generation in generations[-num_bounce_generations:]: generation = int(generation) if stable_generation == -1: stable_generation = generation assert stable_generation == generation, \ "Stream rolling bounce have caused unexpected generation bump %d" % generation self.verify_processing(processors) stop_processors(processors, self.stopped_message) self.producer.stop() self.kafka.stop() self.zookeeper.stop() def verify_processing(self, processors): for processor in processors: with processor.node.account.monitor_log( processor.STDOUT_FILE) as monitor: monitor.wait_until( self.pattern, timeout_sec=60, err_msg="Never saw processing of %s " % self.pattern + str(processor.node.account)) def set_topics(self, processor): processor.INPUT_TOPIC = self.input_topic
class ReplicaVerificationToolTest(Test): """ Tests ReplicaVerificationTool """ def __init__(self, test_context): super(ReplicaVerificationToolTest, self).__init__(test_context) self.num_zk = 1 self.num_brokers = 2 self.messages_received_count = 0 self.topics = {TOPIC: {'partitions': 1, 'replication-factor': 2}} self.zk = ZookeeperService(test_context, self.num_zk) self.kafka = None self.producer = None self.replica_verifier = None def setUp(self): self.zk.start() def start_kafka(self, security_protocol, interbroker_security_protocol): self.kafka = KafkaService( self.test_context, self.num_brokers, self.zk, security_protocol=security_protocol, interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) self.kafka.start() def start_replica_verification_tool(self, security_protocol): self.replica_verifier = ReplicaVerificationTool( self.test_context, 1, self.kafka, TOPIC, report_interval_ms=REPORT_INTERVAL_MS, security_protocol=security_protocol) self.replica_verifier.start() def start_producer(self, max_messages, acks, timeout): # This will produce to kafka cluster current_acked = 0 self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, acks=acks, max_messages=max_messages) self.producer.start() wait_until( lambda: acks == 0 or self.producer.num_acked >= current_acked + max_messages, timeout_sec=timeout, err_msg="Timeout awaiting messages to be produced and acked") def stop_producer(self): self.producer.stop() @cluster(num_nodes=6) def test_replica_lags(self, security_protocol='PLAINTEXT'): """ Tests ReplicaVerificationTool :return: None """ self.start_kafka(security_protocol, security_protocol) self.start_replica_verification_tool(security_protocol) self.start_producer(max_messages=10, acks=-1, timeout=15) # Verify that there is no lag in replicas and is correctly reported by ReplicaVerificationTool wait_until( lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) == 0, timeout_sec=10, err_msg="Timed out waiting to reach zero replica lags.") self.stop_producer() self.start_producer(max_messages=10000, acks=0, timeout=5) # Verify that there is lag in replicas and is correctly reported by ReplicaVerificationTool wait_until( lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) > 0, timeout_sec=10, err_msg= "Timed out waiting to reach non-zero number of replica lags.")
class TestBounce(Test): """Sanity checks on verifiable producer service class with cluster roll.""" def __init__(self, test_context): super(TestBounce, self).__init__(test_context) quorum_size_arg_name = 'quorum_size' default_quorum_size = 1 quorum_size = default_quorum_size if not test_context.injected_args else test_context.injected_args.get( quorum_size_arg_name, default_quorum_size) if quorum_size < 1: raise Exception("Illegal %s value provided for the test: %s" % (quorum_size_arg_name, quorum_size)) self.topic = "topic" self.zk = ZookeeperService(test_context, num_nodes=quorum_size) if quorum.for_test( test_context) == quorum.zk else None num_kafka_nodes = quorum_size if quorum.for_test( test_context) == quorum.colocated_kraft else 1 self.kafka = KafkaService( test_context, num_nodes=num_kafka_nodes, zk=self.zk, topics={self.topic: { "partitions": 1, "replication-factor": 1 }}, controller_num_nodes_override=quorum_size) self.num_messages = 1000 def create_producer(self): # This will produce to source kafka cluster self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=self.topic, max_messages=self.num_messages, throughput=self.num_messages // 10) def setUp(self): if self.zk: self.zk.start() # ZooKeeper and KRaft, quorum size = 1 @cluster(num_nodes=4) @matrix(metadata_quorum=quorum.all, quorum_size=[1]) # Remote and Co-located KRaft, quorum size = 3 @cluster(num_nodes=6) @matrix(metadata_quorum=quorum.all_kraft, quorum_size=[3]) def test_simple_run(self, metadata_quorum, quorum_size): """ Test that we can start VerifiableProducer on the current branch snapshot version, and verify that we can produce a small number of messages both before and after a subsequent roll. """ self.kafka.start() for first_time in [True, False]: self.create_producer() self.producer.start() wait_until( lambda: self.producer.num_acked > 5, timeout_sec=15, err_msg= "Producer failed to start in a reasonable amount of time.") self.producer.wait() num_produced = self.producer.num_acked assert num_produced == self.num_messages, "num_produced: %d, num_messages: %d" % ( num_produced, self.num_messages) if first_time: self.producer.stop() if self.kafka.quorum_info.using_kraft and self.kafka.remote_controller_quorum: self.kafka.remote_controller_quorum.restart_cluster() self.kafka.restart_cluster()
class StreamsOptimizedTest(Test): """ Test doing upgrades of a Kafka Streams application that is un-optimized initially then optimized """ input_topic = 'inputTopic' aggregation_topic = 'aggregationTopic' reduce_topic = 'reduceTopic' join_topic = 'joinTopic' operation_pattern = 'AGGREGATED\|REDUCED\|JOINED' def __init__(self, test_context): super(StreamsOptimizedTest, self).__init__(test_context) self.topics = { self.input_topic: {'partitions': 6}, self.aggregation_topic: {'partitions': 6}, self.reduce_topic: {'partitions': 6}, self.join_topic: {'partitions': 6} } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.input_topic, throughput=1000, acks=1) def test_upgrade_optimized_topology(self): self.zookeeper.start() self.kafka.start() processor1 = StreamsOptimizedUpgradeTestService(self.test_context, self.kafka) processor2 = StreamsOptimizedUpgradeTestService(self.test_context, self.kafka) processor3 = StreamsOptimizedUpgradeTestService(self.test_context, self.kafka) processors = [processor1, processor2, processor3] # produce records continually during the test self.producer.start() # start all processors unoptimized for processor in processors: self.set_topics(processor) processor.CLEAN_NODE_ENABLED = False self.verify_running_repartition_topic_count(processor, 4) self.verify_processing(processors, verify_individual_operations=False) self.stop_processors(processors) # start again with topology optimized for processor in processors: processor.OPTIMIZED_CONFIG = 'all' self.verify_running_repartition_topic_count(processor, 1) self.verify_processing(processors, verify_individual_operations=True) self.stop_processors(processors) self.producer.stop() self.kafka.stop() self.zookeeper.stop() @staticmethod def verify_running_repartition_topic_count(processor, repartition_topic_count): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.start() monitor.wait_until('REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' % repartition_topic_count, timeout_sec=120, err_msg="Never saw 'REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' message " % repartition_topic_count + str(processor.node.account)) @staticmethod def verify_stopped(processor): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.stop() monitor.wait_until('OPTIMIZE_TEST Streams Stopped', timeout_sec=60, err_msg="'OPTIMIZE_TEST Streams Stopped' message" + str(processor.node.account)) def verify_processing(self, processors, verify_individual_operations): for processor in processors: if not self.all_source_subtopology_tasks(processor): if verify_individual_operations: for operation in self.operation_pattern.split('\|'): self.do_verify(processor, operation) else: self.do_verify(processor, self.operation_pattern) else: self.logger.info("Skipping processor %s with all source tasks" % processor.node.account) def do_verify(self, processor, pattern): self.logger.info("Verifying %s processing pattern in STDOUT_FILE" % pattern) with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor: monitor.wait_until(pattern, timeout_sec=60, err_msg="Never saw processing of %s " % pattern + str(processor.node.account)) def all_source_subtopology_tasks(self, processor): retries = 0 while retries < 5: found = list(processor.node.account.ssh_capture("sed -n 's/.*current active tasks: \[\(\(0_[0-9], \)\{3\}0_[0-9]\)\].*/\1/p' %s" % processor.LOG_FILE, allow_fail=True)) self.logger.info("Returned %s from assigned task check" % found) if len(found) > 0: return True retries += 1 time.sleep(1) return False def stop_processors(self, processors): for processor in processors: self.verify_stopped(processor) def set_topics(self, processor): processor.INPUT_TOPIC = self.input_topic processor.AGGREGATION_TOPIC = self.aggregation_topic processor.REDUCE_TOPIC = self.reduce_topic processor.JOIN_TOPIC = self.join_topic
class StreamsNamedRepartitionTopicTest(Test): """ Tests using a named repartition topic by starting application then doing a rolling upgrade with added operations and the application still runs """ input_topic = 'inputTopic' aggregation_topic = 'aggregationTopic' pattern = 'AGGREGATED' def __init__(self, test_context): super(StreamsNamedRepartitionTopicTest, self).__init__(test_context) self.topics = { self.input_topic: { 'partitions': 6 }, self.aggregation_topic: { 'partitions': 6 } } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.input_topic, throughput=1000, acks=1) def test_upgrade_topology_with_named_repartition_topic(self): self.zookeeper.start() self.kafka.start() processor1 = StreamsNamedRepartitionTopicService( self.test_context, self.kafka) processor2 = StreamsNamedRepartitionTopicService( self.test_context, self.kafka) processor3 = StreamsNamedRepartitionTopicService( self.test_context, self.kafka) processors = [processor1, processor2, processor3] self.producer.start() for processor in processors: processor.CLEAN_NODE_ENABLED = False self.set_topics(processor) self.verify_running(processor, 'REBALANCING -> RUNNING') self.verify_processing(processors) # do rolling upgrade for processor in processors: self.verify_stopped(processor) # will tell app to add operations before repartition topic processor.ADD_ADDITIONAL_OPS = 'true' self.verify_running(processor, 'UPDATED Topology') self.verify_processing(processors) self.stop_processors(processors) self.producer.stop() self.kafka.stop() self.zookeeper.stop() @staticmethod def verify_running(processor, message): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.start() monitor.wait_until(message, timeout_sec=60, err_msg="Never saw '%s' message " % message + str(processor.node.account)) @staticmethod def verify_stopped(processor): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.stop() monitor.wait_until( 'NAMED_REPARTITION_TEST Streams Stopped', timeout_sec=60, err_msg="'NAMED_REPARTITION_TEST Streams Stopped' message" + str(processor.node.account)) def verify_processing(self, processors): for processor in processors: with processor.node.account.monitor_log( processor.STDOUT_FILE) as monitor: monitor.wait_until( self.pattern, timeout_sec=60, err_msg="Never saw processing of %s " % self.pattern + str(processor.node.account)) def stop_processors(self, processors): for processor in processors: self.verify_stopped(processor) def set_topics(self, processor): processor.INPUT_TOPIC = self.input_topic processor.AGGREGATION_TOPIC = self.aggregation_topic
class StreamsOptimizedTest(Test): """ Test doing upgrades of a Kafka Streams application that is un-optimized initially then optimized """ input_topic = 'inputTopic' aggregation_topic = 'aggregationTopic' reduce_topic = 'reduceTopic' join_topic = 'joinTopic' operation_pattern = 'AGGREGATED\|REDUCED\|JOINED' stopped_message = 'OPTIMIZE_TEST Streams Stopped' def __init__(self, test_context): super(StreamsOptimizedTest, self).__init__(test_context) self.topics = { self.input_topic: { 'partitions': 6 }, self.aggregation_topic: { 'partitions': 6 }, self.reduce_topic: { 'partitions': 6 }, self.join_topic: { 'partitions': 6 } } self.zookeeper = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zookeeper, topics=self.topics) self.producer = VerifiableProducer(self.test_context, 1, self.kafka, self.input_topic, throughput=1000, acks=1) def test_upgrade_optimized_topology(self): self.zookeeper.start() self.kafka.start() processor1 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processor2 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processor3 = StreamsOptimizedUpgradeTestService( self.test_context, self.kafka) processors = [processor1, processor2, processor3] self.logger.info("produce records continually during the test") self.producer.start() self.logger.info("start all processors unoptimized") for processor in processors: self.set_topics(processor) processor.CLEAN_NODE_ENABLED = False self.verify_running_repartition_topic_count(processor, 4) self.logger.info("verify unoptimized") self.verify_processing(processors, verify_individual_operations=False) self.logger.info("stop unoptimized") stop_processors(processors, self.stopped_message) self.logger.info("reset") self.reset_application() for processor in processors: processor.node.account.ssh("mv " + processor.LOG_FILE + " " + processor.LOG_FILE + ".1", allow_fail=False) processor.node.account.ssh("mv " + processor.STDOUT_FILE + " " + processor.STDOUT_FILE + ".1", allow_fail=False) processor.node.account.ssh("mv " + processor.STDERR_FILE + " " + processor.STDERR_FILE + ".1", allow_fail=False) processor.node.account.ssh("mv " + processor.CONFIG_FILE + " " + processor.CONFIG_FILE + ".1", allow_fail=False) self.logger.info("start again with topology optimized") for processor in processors: processor.OPTIMIZED_CONFIG = 'all' self.verify_running_repartition_topic_count(processor, 1) self.logger.info("verify optimized") self.verify_processing(processors, verify_individual_operations=True) self.logger.info("stop optimized") stop_processors(processors, self.stopped_message) self.logger.info("teardown") self.producer.stop() self.kafka.stop() self.zookeeper.stop() def reset_application(self): resetter = StreamsResetter(self.test_context, self.kafka, topic=self.input_topic, applicationId='StreamsOptimizedTest') resetter.start() # resetter is not long-term running but it would be better to check the pid by stopping it resetter.stop() @staticmethod def verify_running_repartition_topic_count(processor, repartition_topic_count): node = processor.node with node.account.monitor_log(processor.STDOUT_FILE) as monitor: processor.start() monitor.wait_until( 'REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' % repartition_topic_count, timeout_sec=120, err_msg= "Never saw 'REBALANCING -> RUNNING with REPARTITION TOPIC COUNT=%s' message " % repartition_topic_count + str(processor.node.account)) def verify_processing(self, processors, verify_individual_operations): # This test previously had logic to account for skewed assignments, in which not all processors may # receive active assignments. I don't think this will happen anymore, but keep an eye out if we see # test failures here. If that does resurface, note that the prior implementation was not correct. # A better approach would be to make sure we see processing of each partition across the whole cluster # instead of just expecting to see each node perform some processing. for processor in processors: if verify_individual_operations: for operation in self.operation_pattern.split('\|'): self.do_verify(processor, operation) else: self.do_verify(processor, self.operation_pattern) def do_verify(self, processor, pattern): self.logger.info("Verifying %s processing pattern in STDOUT_FILE" % pattern) self.logger.info( list( processor.node.account.ssh_capture("ls -lh %s" % (processor.STDOUT_FILE), allow_fail=True))) wait_until(lambda: processor.node.account.ssh( "grep --max-count 1 '%s' %s" % (pattern, processor.STDOUT_FILE), allow_fail=True) == 0, timeout_sec=60) def set_topics(self, processor): processor.INPUT_TOPIC = self.input_topic processor.AGGREGATION_TOPIC = self.aggregation_topic processor.REDUCE_TOPIC = self.reduce_topic processor.JOIN_TOPIC = self.join_topic