def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT", interbroker_security_protocol=None, new_consumer=True, num_consumers=1, client_version=str(TRUNK), broker_version=str(TRUNK)): """ Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions (using new consumer iff new_consumer == True), and report throughput. """ client_version = KafkaVersion(client_version) broker_version = KafkaVersion(broker_version) self.validate_versions(client_version, broker_version) if interbroker_security_protocol is None: interbroker_security_protocol = security_protocol self.start_kafka(security_protocol, interbroker_security_protocol, broker_version) num_records = 10 * 1000 * 1000 # 10e6 # seed kafka w/messages self.producer = ProducerPerformanceService( self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, settings={ 'acks': 1, 'compression.type': compression_type, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory }) self.producer.run() # consume self.consumer = ConsumerPerformanceService(self.test_context, num_consumers, self.kafka, topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records) self.consumer.group = "test-consumer-group" self.consumer.run() return compute_aggregate_throughput(self.consumer)
def test_upgrade_brokers(self, from_version, to_version): """ Start a smoke test client then perform rolling upgrades on the broker. """ if from_version != to_version: # Setup phase self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() # number of nodes needs to be >= 3 for the smoke test self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=KafkaVersion(from_version), topics=self.topics) self.kafka.start() # allow some time for topics to be created time.sleep(10) # use the current (dev) version driver self.driver = StreamsSmokeTestDriverService( self.test_context, self.kafka) self.driver.node.version = KafkaVersion(from_version) self.driver.start() self.processor1 = StreamsSmokeTestJobRunnerService( self.test_context, self.kafka) self.processor1.node.version = KafkaVersion(from_version) self.processor1.start() time.sleep(15) self.perform_broker_upgrade(to_version) time.sleep(15) self.driver.wait() self.driver.stop() self.processor1.stop() self.driver.node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % self.driver.STDOUT_FILE, allow_fail=False) self.processor1.node.account.ssh_capture( "grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False)
def test_upgrade(self, from_kafka_version, to_message_format_version, compression_types, new_consumer=True, security_protocol="PLAINTEXT"): """Test upgrade of Kafka broker cluster from 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2 to the current version from_kafka_version is a Kafka version to upgrade from If to_message_format_version is None, it means that we will upgrade to default (latest) message format version. It is possible to upgrade to 0.10 brokers but still use message format version 0.9 - Start 3 node broker cluster on version 'from_kafka_version' - Start producer and consumer in the background - Perform two-phase rolling upgrade - First phase: upgrade brokers to 0.10 with inter.broker.protocol.version set to from_kafka_version and log.message.format.version set to from_kafka_version - Second phase: remove inter.broker.protocol.version config with rolling bounce; if to_message_format_version is set to 0.9, set log.message.format.version to to_message_format_version, otherwise remove log.message.format.version config - Finally, validate that every message acked by the producer was consumed by the consumer """ self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=KafkaVersion(from_kafka_version), topics={self.topic: {"partitions": 3, "replication-factor": 3, 'configs': {"min.insync.replicas": 2}}}) self.kafka.security_protocol = security_protocol self.kafka.interbroker_security_protocol = security_protocol self.kafka.start() self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput, message_validator=is_int, compression_types=compression_types, version=KafkaVersion(from_kafka_version)) if from_kafka_version <= LATEST_0_10_0: assert self.kafka.cluster_id() is None # TODO - reduce the timeout self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=30000, new_consumer=new_consumer, message_validator=is_int, version=KafkaVersion(from_kafka_version)) self.run_produce_consume_validate(core_test_action=lambda: self.perform_upgrade(from_kafka_version, to_message_format_version)) cluster_id = self.kafka.cluster_id() assert cluster_id is not None assert len(cluster_id) == 22
def test_broker_compatibility(self, broker_version, auto_create_topics, security_protocol, connect_protocol): """ Verify that Connect will start up with various broker versions with various configurations. When Connect distributed starts up, it either creates internal topics (v0.10.1.0 and after) or relies upon the broker to auto-create the topics (v0.10.0.x and before). """ self.CONNECT_PROTOCOL = connect_protocol self.setup_services(broker_version=KafkaVersion(broker_version), auto_create_topics=auto_create_topics, security_protocol=security_protocol) self.cc.set_configs(lambda node: self.render( "connect-distributed.properties", node=node)) self.cc.start() self.logger.info("Creating connectors") self._start_connector("connect-file-source.properties") self._start_connector("connect-file-sink.properties") # Generating data on the source node should generate new records and create new output on the sink node. Timeouts # here need to be more generous than they are for standalone mode because a) it takes longer to write configs, # do rebalancing of the group, etc, and b) without explicit leave group support, rebalancing takes awhile for node in self.cc.nodes: node.account.ssh("echo -e -n " + repr(self.FIRST_INPUTS) + " >> " + self.INPUT_FILE) wait_until( lambda: self._validate_file_output(self.FIRST_INPUT_LIST), timeout_sec=70, err_msg= "Data added to input file was not seen in the output file in a reasonable amount of time." )
def test_simple_run(self, producer_version=DEV_BRANCH): """ Test that we can start VerifiableProducer on the current branch snapshot version or against the 0.8.2 jar, and verify that we can produce a small number of messages. """ node = self.producer.nodes[0] node.version = KafkaVersion(producer_version) self.producer.start() wait_until( lambda: self.producer.num_acked > 5, timeout_sec=5, err_msg="Producer failed to start in a reasonable amount of time.") # using version.vstring (distutils.version.LooseVersion) is a tricky way of ensuring # that this check works with DEV_BRANCH # When running VerifiableProducer 0.8.X, both the current branch version and 0.8.X should show up because of the # way verifiable producer pulls in some development directories into its classpath # # If the test fails here because 'ps .. | grep' couldn't find the process it means # the login and grep that is_version() performs is slower than # the time it takes the producer to produce its messages. # Easy fix is to decrease throughput= above, the good fix is to make the producer # not terminate until explicitly killed in this case. if node.version <= LATEST_0_8_2: assert is_version(node, [node.version.vstring, DEV_BRANCH.vstring], logger=self.logger) else: assert is_version(node, [node.version.vstring], logger=self.logger) 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)
def test_multiple_kraft_sasl_mechanisms(self, metadata_quorum): """ Test for remote KRaft cases that we can start VerifiableProducer on the current branch snapshot version, and verify that we can produce a small number of messages. The inter-controller and broker-to-controller security protocols are both SASL_PLAINTEXT but the SASL mechanisms are different (we set GSSAPI for the inter-controller mechanism and PLAIN for the broker-to-controller mechanism). This test differs from the above tests -- he ones above used the same SASL mechanism for both paths. """ self.kafka.security_protocol = self.kafka.interbroker_security_protocol = 'PLAINTEXT' controller_quorum = self.kafka.controller_quorum controller_quorum.controller_security_protocol = 'SASL_PLAINTEXT' controller_quorum.controller_sasl_mechanism = 'PLAIN' controller_quorum.intercontroller_security_protocol = 'SASL_PLAINTEXT' controller_quorum.intercontroller_sasl_mechanism = 'GSSAPI' self.kafka.start() node = self.producer.nodes[0] node.version = KafkaVersion(str(DEV_BRANCH)) 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.") # See above comment above regarding use of version.vstring (distutils.version.LooseVersion) assert is_version(node, [node.version.vstring], logger=self.logger) 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)
def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() features = get_broker_features(broker_version) self.invoke_compatibility_program(features)
def test_simple_run(self, producer_version=DEV_BRANCH): """ Test that we can start VerifiableProducer on the current branch snapshot version or against the 0.8.2 jar, and verify that we can produce a small number of messages. """ node = self.producer.nodes[0] node.version = KafkaVersion(producer_version) self.producer.start() wait_until( lambda: self.producer.num_acked > 5, timeout_sec=5, err_msg="Producer failed to start in a reasonable amount of time.") # using version.vstring (distutils.version.LooseVersion) is a tricky way of ensuring # that this check works with DEV_BRANCH # When running VerifiableProducer 0.8.X, both the current branch version and 0.8.X should show up because of the # way verifiable producer pulls in some development directories into its classpath if node.version <= LATEST_0_8_2: assert is_version(node, [node.version.vstring, DEV_BRANCH.vstring]) else: assert is_version(node, [node.version.vstring]) 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)
def test_fail_fast_on_incompatible_brokers_if_eos_beta_enabled( self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "exactly_once_beta") with processor.node.account.monitor_log( processor.STDERR_FILE) as monitor: with processor.node.account.monitor_log(processor.LOG_FILE) as log: processor.start() log.wait_until( 'Shutting down because the Kafka cluster seems to be on a too old version. Setting processing\.guarantee="exactly_once_beta" requires broker version 2\.5 or higher\.', timeout_sec=60, err_msg= "Never saw 'Shutting down, because the Kafka cluster seems to be on a too old version. Setting `processing.guarantee=\"exaclty_once_beta\"` requires broker version 2.5 or higher.' log message " + str(processor.node.account)) monitor.wait_until( 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException', timeout_sec=60, err_msg= "Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException' error message " + str(processor.node.account)) self.kafka.stop()
def test_upgrade_and_downgrade(self, version, compression_types, security_protocol="PLAINTEXT"): """Test upgrade and downgrade of Kafka cluster from old versions to the current version `version` is the Kafka version to upgrade from and downgrade back to Downgrades are supported to any version which is at or above the current `inter.broker.protocol.version` (IBP). For example, if a user upgrades from 1.1 to 2.3, but they leave the IBP set to 1.1, then downgrading to any version at 1.1 or higher is supported. This test case verifies that producers and consumers continue working during the course of an upgrade and downgrade. - Start 3 node broker cluster on version 'kafka_version' - Start producer and consumer in the background - Roll the cluster to upgrade to the current version with IBP set to 'kafka_version' - Roll the cluster to downgrade back to 'kafka_version' - Finally, validate that every message acked by the producer was consumed by the consumer """ kafka_version = KafkaVersion(version) self.setup_services(kafka_version, compression_types, security_protocol) self.await_startup() self.logger.info("First pass bounce - rolling upgrade") self.upgrade_from(kafka_version) self.run_validation() self.logger.info("Second pass bounce - rolling downgrade") self.downgrade_to(kafka_version) self.run_validation()
def test_produce_consume(self, broker_version): print("running producer_consumer_compat with broker_version = %s" % broker_version) self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.security_protocol = "PLAINTEXT" self.kafka.interbroker_security_protocol = self.kafka.security_protocol self.producer = VerifiableProducer( self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput, message_validator=is_int_with_prefix) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=60000, message_validator=is_int_with_prefix) self.kafka.start() self.run_produce_consume_validate(lambda: wait_until( lambda: self.producer.each_produced_at_least( self.messages_per_producer) == True, timeout_sec=120, backoff_sec=1, err_msg= "Producer did not produce all messages in reasonable amount of time" ))
def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, compression_type="none", security_protocol='PLAINTEXT', tls_version=None, client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor, security protocol and message size are varied depending on arguments injected into this test. Collect and return aggregate throughput statistics after all messages have been acknowledged. (This runs ProducerPerformance.java under the hood) """ client_version = KafkaVersion(client_version) broker_version = KafkaVersion(broker_version) self.validate_versions(client_version, broker_version) self.start_kafka(security_protocol, security_protocol, broker_version, tls_version) # Always generate the same total amount of data nrecords = int(self.target_data_size / message_size) self.producer = ProducerPerformanceService(self.test_context, num_producers, self.kafka, topic=topic, num_records=nrecords, record_size=message_size, throughput=-1, version=client_version, settings={ 'acks': acks, 'compression.type': compression_type, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory }) self.producer.run() return compute_aggregate_throughput(self.producer)
def test_producer_and_consumer(self, compression_type="none", security_protocol="PLAINTEXT", tls_version=None, interbroker_security_protocol=None, client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Concurrently produce and consume 10e6 messages with a single producer and a single consumer, Return aggregate throughput statistics for both producer and consumer. (Under the hood, this runs ProducerPerformance.java, and ConsumerPerformance.scala) """ client_version = KafkaVersion(client_version) broker_version = KafkaVersion(broker_version) self.validate_versions(client_version, broker_version) if interbroker_security_protocol is None: interbroker_security_protocol = security_protocol self.start_kafka(security_protocol, interbroker_security_protocol, broker_version, tls_version) num_records = 10 * 1000 * 1000 # 10e6 self.producer = ProducerPerformanceService( self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, settings={ 'acks': 1, 'compression.type': compression_type, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory } ) self.consumer = ConsumerPerformanceService( self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, messages=num_records) Service.run_parallel(self.producer, self.consumer) data = { "producer": compute_aggregate_throughput(self.producer), "consumer": compute_aggregate_throughput(self.consumer) } summary = [ "Producer + consumer:", str(data)] self.logger.info("\n".join(summary)) return data
def test_upgrade_downgrade_streams(self, from_version, to_version): """ Start a smoke test client, then abort (kill -9) and restart it a few times. Ensure that all records are delivered. Note, that just like tests/core/upgrade_test.py, a prerequisite for this test to succeed if the inclusion of all parametrized versions of kafka in kafka/vagrant/base.sh (search for get_kafka()). For streams in particular, that means that someone has manually copies the kafka-stream-$version-test.jar in the right S3 bucket as shown in base.sh. """ if from_version != to_version: # Setup phase self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() # number of nodes needs to be >= 3 for the smoke test self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=KafkaVersion(from_version), topics=self.topics) self.kafka.start() # allow some time for topics to be created time.sleep(10) self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.node.version = KafkaVersion(from_version) self.driver.start() self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka) self.processor1.node.version = KafkaVersion(from_version) self.processor1.start() time.sleep(15) self.perform_streams_upgrade(to_version) time.sleep(15) self.driver.wait() self.driver.stop() self.processor1.stop() self.driver.node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % self.driver.STDOUT_FILE, allow_fail=False) self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False)
def perform_streams_upgrade(self, to_version): self.logger.info("First pass bounce - rolling streams upgrade") # get the node running the streams app node = self.processor1.node self.processor1.stop() # change it's version. This will automatically make it pick up a different # JAR when it starts again node.version = KafkaVersion(to_version) self.processor1.start()
def test_rolling_upgrade_downgrade_multiple_apps(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() # verification step run after each upgrade self.update_processors_and_verify(self.streams_upgrade_versions) # with order reversed now we test downgrading, verification run after each downgrade self.update_processors_and_verify(self.streams_downgrade_versions) for processor in self.processors: self.stop(processor)
def prop_file(self): properties = self.extra_properties.copy() properties[streams_property.STATE_DIR] = self.PERSISTENT_ROOT properties[ streams_property.KAFKA_SERVERS] = self.kafka.bootstrap_servers() if self.UPGRADE_FROM is not None: properties['upgrade.from'] = self.UPGRADE_FROM if (self.UPGRADE_FROM is not None and KafkaVersion(self.UPGRADE_FROM).supports_fk_joins()) or \ (self.KAFKA_STREAMS_VERSION is not None and KafkaVersion(self.KAFKA_STREAMS_VERSION).supports_fk_joins()): properties['test.run_fk_join'] = "true" if self.UPGRADE_TO == "future_version": properties['test.future.metadata'] = "any_value" # Long.MAX_VALUE lets us do the assignment without a warmup properties['acceptable.recovery.lag'] = "9223372036854775807" properties[ "session.timeout.ms"] = "10000" # set back to 10s for tests. See KIP-735 cfg = KafkaConfig(**properties) return cfg.render()
def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() features = get_broker_features(broker_version) if not self.zk: # The self-managed mode doesn't support acls yet, we should remove this once it does features["describe-acls-supported"] = False self.invoke_compatibility_program(features)
def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() features = get_broker_features(broker_version) if not self.zk: # this check/disabling is only necessary due to the fact that we are in early access mode with # KIP-500 and we should remove the special casing when that his fully implemented features["describe-acls-supported"] = False self.invoke_compatibility_program(features)
def test_fail_fast_on_incompatible_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "at_least_once") with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: processor.start() monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException', timeout_sec=60, err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException " + str(processor.node.account)) self.kafka.stop()
def test_fail_fast_on_incompatible_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False) processor.start() processor.node.account.ssh(processor.start_cmd(processor.node)) with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.StreamsException: Kafka Streams requires broker version 0.10.1.x or higher.', timeout_sec=60, err_msg="Never saw 'Streams requires broker verion 0.10.1+' error message " + str(processor.node.account)) self.kafka.stop()
def test_fail_fast_on_incompatible_brokers_if_eos_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, True) processor.start() processor.node.account.ssh(processor.start_cmd(processor.node)) with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.StreamsException: Setting processing.guarantee=exactly_once requires broker version 0.11.0.x or higher.', timeout_sec=60, err_msg="Never saw 'EOS requires broker version 0.11+' error message " + str(processor.node.account)) self.kafka.stop()
def test_timeout_on_pre_010_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False) processor.start() processor.node.account.ssh(processor.start_cmd(processor.node)) with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.BrokerNotFoundException: Could not find any available broker.', timeout_sec=60, err_msg="Never saw 'no available brokers' error message " + str(processor.node.account)) self.kafka.stop()
def test_fail_fast_on_incompatible_brokers_if_eos_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, True) processor.start() processor.node.account.ssh(processor.start_cmd(processor.node)) with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS ', timeout_sec=60, err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS ' error message " + str(processor.node.account)) self.kafka.stop()
def test_fail_fast_on_incompatible_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False) processor.start() processor.node.account.ssh(processor.start_cmd(processor.node)) with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.streams.errors.StreamsException: Could not create topic kafka-streams-system-test-broker-compatibility-KSTREAM-AGGREGATE-STATE-STORE-0000000001-changelog.', timeout_sec=60, err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.streams.errors.StreamsException: Could not create topic kafka-streams-system-test-broker-compatibility-KSTREAM-AGGREGATE-STATE-STORE-0000000001-changelog.' error message " + str(processor.node.account)) self.kafka.stop()
def upgrade_and_verify_start(self, processors, to_version): for processor in processors: self.logger.info("Updating node %s to version %s" % (processor.node.account, to_version)) node = processor.node if self.started: self.stop(processor) node.version = KafkaVersion(to_version) processor.start() self.wait_for_verification(processor, "initializing processor: topic", processor.STDOUT_FILE) self.started = True
def test_compatible_brokers_eos_disabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "at_least_once") processor.start() self.consumer.start() processor.wait() wait_until(lambda: self.consumer.total_consumed() > 0, timeout_sec=30, err_msg="Did expect to read a message but got none within 30 seconds.") self.consumer.stop() self.kafka.stop()
def test_compatible_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() self.processor.start() self.consumer.start() self.processor.wait() num_consumed_mgs = self.consumer.total_consumed() self.consumer.stop() self.kafka.stop() assert num_consumed_mgs == 1, \ "Did expect to read exactly one message but got %d" % num_consumed_mgs
def test_compatible_brokers(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() self.processor.start() self.consumer.start() self.processor.wait() wait_until( lambda: self.consumer.total_consumed() > 0, timeout_sec=30, err_msg= "Did expect to read a message but got none within 30 seconds.") self.consumer.stop() self.kafka.stop()
def test_version(self, version=str(LATEST_0_9), new_consumer=True, metadata_quorum=quorum.zk): """ Sanity check out producer performance service - verify that we can run the service with a small number of messages. The actual stats here are pretty meaningless since the number of messages is quite small. """ version = KafkaVersion(version) self.kafka = KafkaService( self.test_context, 1, self.zk, topics={self.topic: {'partitions': 1, 'replication-factor': 1}}, version=version) self.kafka.start() # check basic run of producer performance self.producer_perf = ProducerPerformanceService( self.test_context, 1, self.kafka, topic=self.topic, num_records=self.num_records, record_size=self.record_size, throughput=1000000000, # Set impossibly for no throttling for equivalent behavior between 0.8.X and 0.9.X version=version, settings={ 'acks': 1, 'batch.size': 8*1024, 'buffer.memory': 64*1024*1024}) self.producer_perf.run() producer_perf_data = compute_aggregate_throughput(self.producer_perf) assert producer_perf_data['records_per_sec'] > 0 # check basic run of end to end latency self.end_to_end = EndToEndLatencyService( self.test_context, 1, self.kafka, topic=self.topic, num_records=self.num_records, version=version) self.end_to_end.run() end_to_end_data = latency(self.end_to_end.results[0]['latency_50th_ms'], self.end_to_end.results[0]['latency_99th_ms'], self.end_to_end.results[0]['latency_999th_ms']) # check basic run of consumer performance service self.consumer_perf = ConsumerPerformanceService( self.test_context, 1, self.kafka, new_consumer=new_consumer, topic=self.topic, version=version, messages=self.num_records) self.consumer_perf.group = "test-consumer-group" self.consumer_perf.run() consumer_perf_data = compute_aggregate_throughput(self.consumer_perf) assert consumer_perf_data['records_per_sec'] > 0 return { "producer_performance": producer_perf_data, "end_to_end_latency": end_to_end_data, "consumer_performance": consumer_perf_data }