You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by cm...@apache.org on 2022/09/29 05:59:32 UTC
[kafka] 01/01: add unit and integration tests
This is an automated email from the ASF dual-hosted git repository.
cmccabe pushed a commit to branch KAFKA-14265
in repository https://gitbox.apache.org/repos/asf/kafka.git
commit fc786c335c6f51a9022b0190b15f57f42e4ea9ab
Author: Colin P. McCabe <cm...@apache.org>
AuthorDate: Wed Sep 28 22:58:39 2022 -0700
add unit and integration tests
---
.../kafka/api/AuthorizerIntegrationTest.scala | 15 +++++++++++-
.../authorizer/StandardAuthorizerData.java | 2 +-
.../authorizer/StandardAuthorizerTest.java | 27 ++++++++++++++++++++++
3 files changed, 42 insertions(+), 2 deletions(-)
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index a109ae8ce4c..d6f736055b2 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -25,7 +25,7 @@ import kafka.security.authorizer.AclEntry.WildcardHost
import kafka.server.{BaseRequestTest, KafkaConfig}
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.waitUntilTrue
-import org.apache.kafka.clients.admin.{Admin, AlterConfigOp}
+import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic}
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
import org.apache.kafka.clients.producer._
@@ -2619,4 +2619,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
)
}
+ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+ @ValueSource(strings = Array("zk", "kraft"))
+ def testPrefixAcls(quorum: String): Unit = {
+ addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, CREATE, ALLOW)),
+ new ResourcePattern(TOPIC, "f", PREFIXED))
+ addAndVerifyAcls(Set(new AccessControlEntry("User:otherPrincipal", WildcardHost, CREATE, DENY)),
+ new ResourcePattern(TOPIC, "fooa", PREFIXED))
+ addAndVerifyAcls(Set(new AccessControlEntry("User:otherPrincipal", WildcardHost, CREATE, ALLOW)),
+ new ResourcePattern(TOPIC, "foob", PREFIXED))
+ val future = createAdminClient().createTopics(Collections.
+ singletonList(new NewTopic("foobar", 1, 1.toShort))).all()
+ JTestUtils.assertFutureThrows(future, classOf[TopicAuthorizationException])
+ }
}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java
index b52a02562fe..6e9efc3cd5d 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java
@@ -656,4 +656,4 @@ public class StandardAuthorizerData {
HashMap<Uuid, StandardAcl> getAclsById() {
return aclsById;
}
-}
+}
\ No newline at end of file
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java
index 987c00155c4..a26eb3d50b5 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java
@@ -640,4 +640,31 @@ public class StandardAuthorizerTest {
assertTrue(futures.get(CONTROLLER).toCompletableFuture().isDone());
assertFalse(futures.get(CONTROLLER).toCompletableFuture().isCompletedExceptionally());
}
+
+ @Test
+ public void testPrefixAcls() throws Exception {
+ StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer();
+ List<StandardAcl> acls = Arrays.asList(
+ new StandardAcl(TOPIC, "fooa", PREFIXED, "User:alice", "*", ALL, ALLOW),
+ new StandardAcl(TOPIC, "foobar", LITERAL, "User:bob", "*", ALL, ALLOW),
+ new StandardAcl(TOPIC, "f", PREFIXED, "User:bob", "*", ALL, ALLOW)
+ );
+ acls.forEach(acl -> {
+ StandardAclWithId aclWithId = withId(acl);
+ authorizer.addAcl(aclWithId.id(), aclWithId.acl());
+ });
+ assertEquals(Arrays.asList(ALLOWED, DENIED, ALLOWED), authorizer.authorize(
+ newRequestContext("bob"),
+ Arrays.asList(
+ newAction(WRITE, TOPIC, "foobarr"),
+ newAction(READ, TOPIC, "goobar"),
+ newAction(READ, TOPIC, "fooa"))));
+
+ assertEquals(Arrays.asList(ALLOWED, DENIED, DENIED), authorizer.authorize(
+ newRequestContext("alice"),
+ Arrays.asList(
+ newAction(DESCRIBE, TOPIC, "fooa"),
+ newAction(WRITE, TOPIC, "bar"),
+ newAction(READ, TOPIC, "baz"))));
+ }
}