class ZooKeeperSecurityUpgradeTest(ProduceConsumeValidateTest): """Tests a rolling upgrade for zookeeper. """ def __init__(self, test_context): super(ZooKeeperSecurityUpgradeTest, self).__init__(test_context=test_context) def setUp(self): self.topic = "test_topic" self.group = "group" self.producer_throughput = 100 self.num_producers = 1 self.num_consumers = 1 self.acls = ACLs(self.test_context) self.zk = ZookeeperService(self.test_context, num_nodes=3) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={ self.topic: { "partitions": 3, "replication-factor": 3, 'configs': { "min.insync.replicas": 2 } } }) def create_producer_and_consumer(self): 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=60000, message_validator=is_int) self.consumer.group_id = self.group @property def no_sasl(self): return self.kafka.security_protocol == "PLAINTEXT" or self.kafka.security_protocol == "SSL" @property def is_secure(self): return self.kafka.security_protocol == "SASL_PLAINTEXT" \ or self.kafka.security_protocol == "SSL" \ or self.kafka.security_protocol == "SASL_SSL" def run_zk_migration(self): # change zk config (auth provider + jaas login) self.zk.zk_sasl = True if self.no_sasl: self.kafka.start_minikdc_if_necessary(self.zk.zk_principals) # restart zk self.zk.restart_cluster() # restart broker with jaas login self.kafka.restart_cluster() # run migration tool for node in self.zk.nodes: self.zk.zookeeper_migration(node, "secure") # restart broker with zookeeper.set.acl=true and acls self.kafka.zk_set_acl = True self.kafka.restart_cluster() @cluster(num_nodes=9) @matrix( security_protocol=["PLAINTEXT", "SSL", "SASL_SSL", "SASL_PLAINTEXT"]) def test_zk_security_upgrade(self, security_protocol): self.zk.start() self.kafka.security_protocol = security_protocol self.kafka.interbroker_security_protocol = security_protocol # set acls if self.is_secure: self.kafka.authorizer_class_name = KafkaService.ACL_AUTHORIZER # Force use of direct ZooKeeper access because Kafka is not yet started self.acls.set_acls( security_protocol, self.kafka, self.topic, self.group, force_use_zk_connection=True, additional_cluster_operations_to_grant=['Create']) if self.no_sasl: self.kafka.start() else: self.kafka.start(self.zk.zk_principals) #Create Producer and Consumer self.create_producer_and_consumer() #Run upgrade self.run_produce_consume_validate(self.run_zk_migration)
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 ZooKeeperAuthorizerTest(Test): """Tests that the ZooKeeper-based Authorizer works wth both ZooKeeper-based and KRaft clusters. Alters client quotas, making sure it works. Rolls Kafka with an authorizer. Alters client quotas, making sure it fails. Adds ACLs with super-user broker credentials. Alters client quotas, making sure it now works again. Removes ACLs. Note that we intend to have separate test explicitly for the KRaft-based replacement for the ZooKeeper-based authorizer. """ def __init__(self, test_context): super(ZooKeeperAuthorizerTest, self).__init__(test_context=test_context) self.topic = "test_topic" # setup ZooKeeper even with KRaft self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk, topics={self.topic: {"partitions": 1, "replication-factor": 1}}, controller_num_nodes_override=1, allow_zk_with_kraft=True) def setUp(self): # start ZooKeeper even with KRaft self.zk.start() self.acls = ACLs(self.test_context) @cluster(num_nodes=4) @matrix(metadata_quorum=quorum.all_non_upgrade) def test_authorizer(self, metadata_quorum): broker_security_protocol = "SSL" broker_principal = "User:CN=systemtest" client_security_protocol = "SASL_PLAINTEXT" client_sasl_mechanism = 'GSSAPI' client_principal = "User:client" self.kafka.interbroker_security_protocol = broker_security_protocol self.kafka.security_protocol = client_security_protocol self.kafka.client_sasl_mechanism = client_sasl_mechanism if self.kafka.quorum_info.using_kraft: controller_quorum = self.kafka.controller_quorum controller_quorum.controller_security_protocol = broker_security_protocol controller_quorum.intercontroller_security_protocol = broker_security_protocol self.kafka.start() # alter client quotas node = self.kafka.nodes[0] alter_client_quotas_cmd = "%s --entity-name foo --entity-type clients --alter --add-config consumer_byte_rate=10000" % \ (self.kafka.kafka_configs_cmd_with_optional_security_settings(node, force_use_zk_connection=False)) alter_client_quotas_cmd_log_msg = "Running alter client quotas command with client/non-broker credentials...\n%s" % alter_client_quotas_cmd self.logger.info(alter_client_quotas_cmd_log_msg) node.account.ssh(alter_client_quotas_cmd) # set authorizer, restart with broker as super user if (metadata_quorum == quorum.remote_kraft): # we need to explicitly reconfigure/restart any remote controller quorum self.kafka.logger.info("Restarting Remote KRaft Controller with authorizer and broker principal as super user") controller_quorum = self.kafka.controller_quorum controller_quorum.authorizer_class_name = KafkaService.ACL_AUTHORIZER controller_quorum.server_prop_overrides = [["super.users", broker_principal]] # for broker to work with an authorizer controller_quorum.restart_cluster() self.kafka.logger.info("Restarting Kafka with authorizer and broker principal as super user") self.kafka.authorizer_class_name = KafkaService.ACL_AUTHORIZER self.kafka.server_prop_overrides = [["super.users", broker_principal]] # for broker to work with an authorizer self.kafka.restart_cluster() # the alter client quotas command should now fail try: self.logger.info("Expecting this to fail: %s" % alter_client_quotas_cmd_log_msg) node.account.ssh(alter_client_quotas_cmd) raise Exception("Expected alter client quotas command to fail with an authorization error, but it succeeded") except RemoteCommandError as e: if "ClusterAuthorizationException: Cluster authorization failed." not in str(e): self.logger.error("Expected alter client quotas command to fail with an authorization error, but it failed with some other issue") raise e self.logger.info("alter client quotas command failed with an authorization error as expected") # add ACLs self.logger.info("Adding ACLs with broker credentials so that alter client quotas command will succeed") self.acls.add_cluster_acl(self.kafka, client_principal, force_use_zk_connection=False, additional_cluster_operations_to_grant=['AlterConfigs'], security_protocol=broker_security_protocol) # the alter client quotas command should now succeed again self.logger.info(alter_client_quotas_cmd_log_msg) node.account.ssh(alter_client_quotas_cmd) # remove ACLs self.logger.info("Removing ACLs with broker credentials so ensure this operation works") self.acls.remove_cluster_acl(self.kafka, client_principal, additional_cluster_operations_to_remove=['AlterConfigs'], security_protocol=broker_security_protocol)
class TestSecurityRollingUpgrade(ProduceConsumeValidateTest): """Tests a rolling upgrade from PLAINTEXT to a secured cluster """ def __init__(self, test_context): super(TestSecurityRollingUpgrade, self).__init__(test_context=test_context) def setUp(self): self.acls = ACLs(self.test_context) self.topic = "test_topic" self.group = "group" self.producer_throughput = 100 self.num_producers = 1 self.num_consumers = 1 self.zk = ZookeeperService(self.test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={ self.topic: { "partitions": 3, "replication-factor": 3, 'configs': { "min.insync.replicas": 2 } } }) self.zk.start() def create_producer_and_consumer(self): 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=60000, message_validator=is_int) self.consumer.group_id = "group" def bounce(self): self.kafka.start_minikdc_if_necessary() self.kafka.restart_cluster( after_each_broker_restart=lambda: time.sleep(10)) def roll_in_secured_settings(self, client_protocol, broker_protocol): # Roll cluster to include inter broker security protocol. self.kafka.setup_interbroker_listener(broker_protocol) self.bounce() # Roll cluster to disable PLAINTEXT port self.kafka.close_port(SecurityConfig.PLAINTEXT) self.set_authorizer_and_bounce(client_protocol, broker_protocol) def set_authorizer_and_bounce(self, client_protocol, broker_protocol): self.kafka.authorizer_class_name = KafkaService.ZK_ACL_AUTHORIZER # Force use of direct ZooKeeper access due to SecurityDisabledException: No Authorizer is configured on the broker. self.acls.set_acls(client_protocol, self.kafka, self.topic, self.group, force_use_zk_connection=True) self.acls.set_acls(broker_protocol, self.kafka, self.topic, self.group, force_use_zk_connection=True) self.bounce() # enables the authorizer def open_secured_port(self, client_protocol): self.kafka.security_protocol = client_protocol self.kafka.open_port(client_protocol) self.kafka.start_minikdc_if_necessary() self.bounce() def add_sasl_mechanism(self, new_client_sasl_mechanism): self.kafka.client_sasl_mechanism = new_client_sasl_mechanism self.kafka.start_minikdc_if_necessary() self.bounce() def roll_in_sasl_mechanism(self, security_protocol, new_sasl_mechanism): # Roll cluster to update inter-broker SASL mechanism. # We need the inter-broker SASL mechanism to still be enabled through this roll. self.kafka.client_sasl_mechanism = "%s,%s" % ( self.kafka.interbroker_sasl_mechanism, new_sasl_mechanism) self.kafka.interbroker_sasl_mechanism = new_sasl_mechanism self.bounce() # Bounce again with ACLs for new mechanism. self.kafka.client_sasl_mechanism = new_sasl_mechanism # Removes old SASL mechanism completely self.set_authorizer_and_bounce(security_protocol, security_protocol) def add_separate_broker_listener(self, broker_security_protocol, broker_sasl_mechanism): # Enable the new internal listener on all brokers first self.kafka.open_port(self.kafka.INTERBROKER_LISTENER_NAME) self.kafka.port_mappings[ self.kafka. INTERBROKER_LISTENER_NAME].security_protocol = broker_security_protocol self.kafka.port_mappings[ self.kafka. INTERBROKER_LISTENER_NAME].sasl_mechanism = broker_sasl_mechanism self.bounce() # Update inter-broker listener after all brokers have been updated to enable the new listener self.kafka.setup_interbroker_listener(broker_security_protocol, True) self.kafka.interbroker_sasl_mechanism = broker_sasl_mechanism self.bounce() def remove_separate_broker_listener(self, client_security_protocol): # This must be done in two phases: keep listening on the INTERNAL listener while rolling once to switch # the inter-broker security listener, then roll again to remove the INTERNAL listener. orig_inter_broker_security_protocol = self.kafka.interbroker_security_protocol self.kafka.setup_interbroker_listener( client_security_protocol, False) # this closes the INTERNAL listener # Re-open the INTERNAL listener self.kafka.open_port(KafkaService.INTERBROKER_LISTENER_NAME) self.kafka.port_mappings[ KafkaService. INTERBROKER_LISTENER_NAME].security_protocol = orig_inter_broker_security_protocol self.kafka.port_mappings[ KafkaService. INTERBROKER_LISTENER_NAME].sasl_mechanism = self.kafka.interbroker_sasl_mechanism self.bounce() # Close the INTERNAL listener for good and bounce again to fully migrate to <client_security_protocol> self.kafka.close_port(KafkaService.INTERBROKER_LISTENER_NAME) self.bounce() @cluster(num_nodes=8) @matrix(client_protocol=[SecurityConfig.SSL]) @cluster(num_nodes=9) @matrix(client_protocol=[ SecurityConfig.SASL_PLAINTEXT, SecurityConfig.SASL_SSL ]) def test_rolling_upgrade_phase_one(self, client_protocol): """ Start with a PLAINTEXT cluster, open a SECURED port, via a rolling upgrade, ensuring we could produce and consume throughout over PLAINTEXT. Finally check we can produce and consume the new secured port. """ self.kafka.setup_interbroker_listener(SecurityConfig.PLAINTEXT) self.kafka.security_protocol = SecurityConfig.PLAINTEXT self.kafka.start() # Create PLAINTEXT producer and consumer self.create_producer_and_consumer() # Rolling upgrade, opening a secure protocol, ensuring the Plaintext producer/consumer continues to run self.run_produce_consume_validate(self.open_secured_port, client_protocol) # Now we can produce and consume via the secured port self.kafka.security_protocol = client_protocol self.create_producer_and_consumer() self.run_produce_consume_validate(lambda: time.sleep(1)) @cluster(num_nodes=8) @matrix(client_protocol=[ SecurityConfig.SASL_SSL, SecurityConfig.SSL, SecurityConfig.SASL_PLAINTEXT ], broker_protocol=[ SecurityConfig.SASL_SSL, SecurityConfig.SSL, SecurityConfig.SASL_PLAINTEXT ]) def test_rolling_upgrade_phase_two(self, client_protocol, broker_protocol): """ Start with a PLAINTEXT cluster with a second Secured port open (i.e. result of phase one). A third secure port is also open if inter-broker and client protocols are different. Start a Producer and Consumer via the SECURED client port Incrementally upgrade to add inter-broker be the secure broker protocol Incrementally upgrade again to add ACLs as well as disabling the PLAINTEXT port Ensure the producer and consumer ran throughout """ #Given we have a broker that has both secure and PLAINTEXT ports open self.kafka.security_protocol = client_protocol self.kafka.setup_interbroker_listener(SecurityConfig.PLAINTEXT, use_separate_listener=False) self.kafka.open_port(broker_protocol) # Set any SASL mechanism explicitly when it isn't already set by the client protocol is_broker_protocol_sasl = broker_protocol in [ SecurityConfig.SASL_SSL, SecurityConfig.SASL_PLAINTEXT ] is_client_protocol_sasl = client_protocol in [ SecurityConfig.SASL_SSL, SecurityConfig.SASL_PLAINTEXT ] if is_broker_protocol_sasl and not is_client_protocol_sasl: self.kafka.port_mappings[ broker_protocol].sasl_mechanism = SecurityConfig.SASL_MECHANISM_GSSAPI self.kafka.start() #Create Secured Producer and Consumer self.create_producer_and_consumer() #Roll in the security protocol. Disable Plaintext. Ensure we can produce and Consume throughout self.run_produce_consume_validate(self.roll_in_secured_settings, client_protocol, broker_protocol) @cluster(num_nodes=9) @matrix(new_client_sasl_mechanism=[SecurityConfig.SASL_MECHANISM_PLAIN]) def test_rolling_upgrade_sasl_mechanism_phase_one( self, new_client_sasl_mechanism): """ Start with a SASL/GSSAPI cluster, add new SASL mechanism, via a rolling upgrade, ensuring we could produce and consume throughout over SASL/GSSAPI. Finally check we can produce and consume using new mechanism. """ self.kafka.setup_interbroker_listener(SecurityConfig.SASL_SSL, use_separate_listener=False) self.kafka.security_protocol = SecurityConfig.SASL_SSL self.kafka.client_sasl_mechanism = SecurityConfig.SASL_MECHANISM_GSSAPI self.kafka.interbroker_sasl_mechanism = SecurityConfig.SASL_MECHANISM_GSSAPI self.kafka.start() # Create SASL/GSSAPI producer and consumer self.create_producer_and_consumer() # Rolling upgrade, adding new SASL mechanism, ensuring the GSSAPI producer/consumer continues to run self.run_produce_consume_validate(self.add_sasl_mechanism, new_client_sasl_mechanism) # Now we can produce and consume using the new SASL mechanism self.kafka.client_sasl_mechanism = new_client_sasl_mechanism self.create_producer_and_consumer() self.run_produce_consume_validate(lambda: time.sleep(1)) @cluster(num_nodes=8) @matrix(new_sasl_mechanism=[SecurityConfig.SASL_MECHANISM_PLAIN]) def test_rolling_upgrade_sasl_mechanism_phase_two(self, new_sasl_mechanism): """ Start with a SASL cluster with GSSAPI for inter-broker and a second mechanism for clients (i.e. result of phase one). Start Producer and Consumer using the second mechanism Incrementally upgrade to set inter-broker to the second mechanism and disable GSSAPI Incrementally upgrade again to add ACLs Ensure the producer and consumer run throughout """ #Start with a broker that has GSSAPI for inter-broker and a second mechanism for clients self.kafka.security_protocol = SecurityConfig.SASL_SSL self.kafka.setup_interbroker_listener(SecurityConfig.SASL_SSL, use_separate_listener=False) self.kafka.client_sasl_mechanism = new_sasl_mechanism self.kafka.interbroker_sasl_mechanism = SecurityConfig.SASL_MECHANISM_GSSAPI self.kafka.start() #Create Producer and Consumer using second mechanism self.create_producer_and_consumer() #Roll in the second SASL mechanism for inter-broker, disabling first mechanism. Ensure we can produce and consume throughout self.run_produce_consume_validate(self.roll_in_sasl_mechanism, self.kafka.security_protocol, new_sasl_mechanism) @cluster(num_nodes=9) def test_enable_separate_interbroker_listener(self): """ Start with a cluster that has a single PLAINTEXT listener. Start producing/consuming on PLAINTEXT port. While doing that, do a rolling restart to enable separate secured interbroker port """ self.kafka.security_protocol = SecurityConfig.PLAINTEXT self.kafka.setup_interbroker_listener(SecurityConfig.PLAINTEXT, use_separate_listener=False) self.kafka.start() self.create_producer_and_consumer() self.run_produce_consume_validate(self.add_separate_broker_listener, SecurityConfig.SASL_SSL, SecurityConfig.SASL_MECHANISM_PLAIN) @cluster(num_nodes=9) def test_disable_separate_interbroker_listener(self): """ Start with a cluster that has two listeners, one on SSL (clients), another on SASL_SSL (broker-to-broker). Start producer and consumer on SSL listener. Close dedicated interbroker listener via rolling restart. Ensure we can produce and consume via SSL listener throughout. """ client_protocol = SecurityConfig.SSL interbroker_security_protocol = SecurityConfig.SASL_SSL interbroker_sasl_mechanism = SecurityConfig.SASL_MECHANISM_GSSAPI self.kafka.security_protocol = client_protocol self.kafka.setup_interbroker_listener(interbroker_security_protocol, use_separate_listener=True) self.kafka.interbroker_sasl_mechanism = interbroker_sasl_mechanism self.kafka.start() # create producer and consumer via client security protocol self.create_producer_and_consumer() # run produce/consume/validate loop while disabling a separate interbroker listener via rolling restart self.run_produce_consume_validate(self.remove_separate_broker_listener, client_protocol)
class ZookeeperTlsTest(ProduceConsumeValidateTest): """Tests TLS connectivity to zookeeper. """ def __init__(self, test_context): super(ZookeeperTlsTest, self).__init__(test_context=test_context) def setUp(self): self.topic = "test_topic" self.group = "group" self.producer_throughput = 100 self.num_producers = 1 self.num_consumers = 1 self.zk = ZookeeperService(self.test_context, num_nodes=3) self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={ self.topic: { "partitions": 3, "replication-factor": 3, 'configs': { "min.insync.replicas": 2 } } }) def create_producer_and_consumer(self): 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=60000, message_validator=is_int) self.consumer.group_id = self.group def perform_produce_consume_validation(self): self.create_producer_and_consumer() self.run_produce_consume_validate() self.producer.free() self.consumer.free() def enable_zk_tls(self): self.test_context.logger.debug( "Enabling the TLS port in Zookeeper (we won't use it from Kafka yet)" ) # change zk config (enable TLS, but also keep non-TLS) self.zk.zk_client_secure_port = True self.zk.restart_cluster() # bounce a Kafka broker -- allows us to detect a broker restart failure as a simple sanity check self.kafka.stop_node(self.kafka.nodes[0]) self.kafka.start_node(self.kafka.nodes[0]) def enable_kafka_zk_tls(self): self.test_context.logger.debug( "Configuring Kafka to use the TLS port in Zookeeper") # change Kafka config (enable TLS to Zookeeper) and restart the Kafka cluster self.kafka.zk_client_secure = True self.kafka.restart_cluster() def disable_zk_non_tls(self): self.test_context.logger.debug( "Disabling the non-TLS port in Zookeeper (as a simple sanity check)" ) # change zk config (disable non-TLS, keep TLS) and restart the ZooKeeper cluster self.zk.zk_client_port = False self.zk.restart_cluster() # bounce a Kafka broker -- allows us to detect a broker restart failure as a simple sanity check self.kafka.stop_node(self.kafka.nodes[0]) self.kafka.start_node(self.kafka.nodes[0]) @cluster(num_nodes=9) def test_zk_tls(self): self.zk.start() self.kafka.security_protocol = self.kafka.interbroker_security_protocol = "PLAINTEXT" self.kafka.start() # Enable TLS port in Zookeeper in addition to the regular non-TLS port # Bounces the ZooKeeper cluster (and a single broker as a sanity check) self.enable_zk_tls() # Leverage ZooKeeper TLS port in Kafka # Bounces the Kafka cluster self.enable_kafka_zk_tls() self.perform_produce_consume_validation() # Disable ZooKeeper non-TLS port to make sure we aren't using it # Bounces the ZooKeeper cluster (and a single broker as a sanity check) self.disable_zk_non_tls() # Make sure the ZooKeeper command line is able to talk to a TLS-enabled ZooKeeper quorum # Test both create() and query(), each of which leverages the ZooKeeper command line # This tests the code in org.apache.zookeeper.ZooKeeperMainWithTlsSupportForKafka path = "/foo" value = "{\"bar\": 0}" self.zk.create(path, value=value) if self.zk.query(path) != value: raise Exception( "Error creating and then querying a znode using the CLI with a TLS-enabled ZooKeeper quorum" ) # Make sure the ConfigCommand CLI is able to talk to a TLS-enabled ZooKeeper quorum # This is necessary for the bootstrap use case despite direct ZooKeeper connectivity being deprecated self.zk.describe(self.topic) # Make sure the AclCommand CLI is able to talk to a TLS-enabled ZooKeeper quorum # This is necessary for the bootstrap use case despite direct ZooKeeper connectivity being deprecated self.zk.list_acls(self.topic) # # Test zookeeper.set.acl with just TLS mutual authentication (no SASL) # # Step 1: run migration tool self.zk.zookeeper_migration(self.zk.nodes[0], "secure") # Step 2: restart brokers with zookeeper.set.acl=true and acls (with TLS but no SASL) self.kafka.zk_set_acl = True self.kafka.restart_cluster() self.perform_produce_consume_validation() # # Test zookeeper.set.acl with both SASL and TLS mutual authentication # # Step 1: remove ACLs created previously self.kafka.zk_set_acl = False self.kafka.restart_cluster() self.zk.zookeeper_migration(self.zk.nodes[0], "unsecure") # Step 2: enable ZooKeeper SASL authentication, but don't take advantage of it in Kafka yet self.zk.zk_sasl = True self.kafka.start_minikdc_if_necessary(self.zk.zk_principals) self.zk.restart_cluster() # bounce a Kafka broker -- allows us to detect a broker restart failure as a simple sanity check self.kafka.stop_node(self.kafka.nodes[0]) self.kafka.start_node(self.kafka.nodes[0]) # Step 3: run migration tool self.zk.zookeeper_migration(self.zk.nodes[0], "secure") # Step 4: restart brokers with zookeeper.set.acl=true and acls (with both TLS and SASL) self.kafka.zk_set_acl = True self.kafka.restart_cluster() self.perform_produce_consume_validation()
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()