You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/05/20 23:35:51 UTC

[GitHub] [kafka] hachikuji opened a new pull request, #12190: KAFKA-13923; Generalize authorizer system test for kraft

hachikuji opened a new pull request, #12190:
URL: https://github.com/apache/kafka/pull/12190

   Now that KRaft supports an authorizer (from KIP-801), we should be covering it in our system tests. I kept around the test with KRaft and the zk `AclAuthorizer` since it seems useful in KRaft to test an implementation which uses an external source.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #12190: KAFKA-13923; Generalize authorizer system test for kraft

Posted by GitBox <gi...@apache.org>.
hachikuji commented on code in PR #12190:
URL: https://github.com/apache/kafka/pull/12190#discussion_r879644296


##########
tests/kafkatest/tests/core/authorizer_test.py:
##########
@@ -36,22 +36,30 @@ class ZooKeeperAuthorizerTest(Test):
     """
 
     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)
+        super(AuthorizerTest, self).__init__(test_context=test_context)
+        self.test_context = test_context
+
     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):
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.KRAFT_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.zk, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    def test_authorizer(self, metadata_quorum, authorizer_class):
+        topics = {"test_topic": {"partitions": 1, "replication-factor": 1}}
+
+        if (authorizer_class == KafkaService.KRAFT_ACL_AUTHORIZER):
+            self.zk = None
+        else:
+            self.zk = ZookeeperService(self.test_context, num_nodes=1)
+            self.zk.start()
+
+        self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk,
+                                  topics=topics, controller_num_nodes_override=1,
+                                  allow_zk_with_kraft=True)
+        self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = True

Review Comment:
   It was mainly so that I could see all the authorizer logs. Perhaps I could change this to only collect after a failure.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on pull request #12190: KAFKA-13923; Generalize authorizer system test for kraft

Posted by GitBox <gi...@apache.org>.
hachikuji commented on PR #12190:
URL: https://github.com/apache/kafka/pull/12190#issuecomment-1133482572

   Successful test run here: http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2022-05-20--001.system-test-kafka-branch-builder--1653090157--hachikuji--KAFKA-13923--017f69bfd3/report.html.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #12190: KAFKA-13923; Generalize authorizer system test for kraft

Posted by GitBox <gi...@apache.org>.
hachikuji commented on code in PR #12190:
URL: https://github.com/apache/kafka/pull/12190#discussion_r879669643


##########
tests/kafkatest/tests/core/authorizer_test.py:
##########
@@ -36,22 +36,30 @@ class ZooKeeperAuthorizerTest(Test):
     """
 
     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)
+        super(AuthorizerTest, self).__init__(test_context=test_context)
+        self.test_context = test_context
+
     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):
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.KRAFT_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.zk, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    def test_authorizer(self, metadata_quorum, authorizer_class):
+        topics = {"test_topic": {"partitions": 1, "replication-factor": 1}}
+
+        if (authorizer_class == KafkaService.KRAFT_ACL_AUTHORIZER):
+            self.zk = None
+        else:
+            self.zk = ZookeeperService(self.test_context, num_nodes=1)
+            self.zk.start()
+
+        self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk,
+                                  topics=topics, controller_num_nodes_override=1,
+                                  allow_zk_with_kraft=True)
+        self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = True

Review Comment:
   Ok, looks like this is unnecessary. Debug logs are collected already after a failure. I will remove.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #12190: KAFKA-13923; Generalize authorizer system test for kraft

Posted by GitBox <gi...@apache.org>.
dajac commented on code in PR #12190:
URL: https://github.com/apache/kafka/pull/12190#discussion_r879408851


##########
tests/kafkatest/tests/core/authorizer_test.py:
##########
@@ -36,22 +36,30 @@ class ZooKeeperAuthorizerTest(Test):
     """
 
     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)
+        super(AuthorizerTest, self).__init__(test_context=test_context)
+        self.test_context = test_context
+
     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):
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.KRAFT_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.remote_kraft, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    @parametrize(metadata_quorum=quorum.zk, authorizer_class=KafkaService.ZK_ACL_AUTHORIZER)
+    def test_authorizer(self, metadata_quorum, authorizer_class):
+        topics = {"test_topic": {"partitions": 1, "replication-factor": 1}}
+
+        if (authorizer_class == KafkaService.KRAFT_ACL_AUTHORIZER):
+            self.zk = None
+        else:
+            self.zk = ZookeeperService(self.test_context, num_nodes=1)
+            self.zk.start()
+
+        self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk,
+                                  topics=topics, controller_num_nodes_override=1,
+                                  allow_zk_with_kraft=True)
+        self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = True

Review Comment:
   Out of curiosity, why are we adding this line? No saying that we should not do it. I just would like to understand the reason.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji merged pull request #12190: KAFKA-13923; Generalize authorizer system test for kraft

Posted by GitBox <gi...@apache.org>.
hachikuji merged PR #12190:
URL: https://github.com/apache/kafka/pull/12190


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org