This is an automated email from the ASF dual-hosted git repository. manikumar pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push: new f3e4dd92293 KAFKA-14827: Support for StandardAuthorizer benchmark (#13423) f3e4dd92293 is described below commit f3e4dd922933bf28b2c091e846cbc4e5255dd1d5 Author: Purshotam Chauhan <pchau...@confluent.io> AuthorDate: Tue Mar 28 14:14:50 2023 +0530 KAFKA-14827: Support for StandardAuthorizer benchmark (#13423) * KAFKA-14827: Support for StandardAuthorizer benchmark Co-authored-by: Purshotam Chauhan <purshotam.r.chau...@gmail.com> * reverting unintentional change --------- Co-authored-by: David Arthur <mum...@gmail.com> Reviewers: Manikumar Reddy <manikumar.re...@gmail.com> --- ...izerBenchmark.java => AuthorizerBenchmark.java} | 70 ++++++++++++++++++---- 1 file changed, 58 insertions(+), 12 deletions(-) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AclAuthorizerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java similarity index 81% rename from jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AclAuthorizerBenchmark.java rename to jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java index 65aa2a1f8d6..c096d634535 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AclAuthorizerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java @@ -20,7 +20,9 @@ package org.apache.kafka.jmh.acl; import kafka.security.authorizer.AclAuthorizer; import kafka.security.authorizer.AclAuthorizer.VersionedAcls; import kafka.security.authorizer.AclEntry; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.acl.AccessControlEntry; +import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclPermissionType; @@ -34,7 +36,10 @@ import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.metadata.authorizer.StandardAcl; +import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.server.authorizer.Action; +import org.apache.kafka.server.authorizer.Authorizer; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -50,6 +55,7 @@ import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; import scala.collection.JavaConverters; +import java.io.IOException; import java.net.InetAddress; import java.util.ArrayList; import java.util.Collections; @@ -61,6 +67,7 @@ import java.util.Random; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; @State(Scope.Benchmark) @Fork(value = 1) @@ -68,7 +75,23 @@ import java.util.concurrent.TimeUnit; @Measurement(iterations = 15) @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) -public class AclAuthorizerBenchmark { +public class AuthorizerBenchmark { + + public enum AuthorizerType { + ACL(AclAuthorizer::new), + KRAFT(StandardAuthorizer::new); + + private Supplier<Authorizer> supplier; + + AuthorizerType(Supplier<Authorizer> supplier) { + this.supplier = supplier; + } + + Authorizer newAuthorizer() { + return supplier.get(); + } + } + @Param({"10000", "50000", "200000"}) private int resourceCount; //no. of. rules per resource @@ -78,10 +101,13 @@ public class AclAuthorizerBenchmark { @Param({"0", "20", "50", "90", "99", "99.9", "99.99", "100"}) private double denyPercentage; + @Param({"ACL", "KRAFT"}) + private AuthorizerType authorizerType; + private final int hostPreCount = 1000; private final String resourceNamePrefix = "foo-bar35_resource-"; - private final AclAuthorizer aclAuthorizer = new AclAuthorizer(); private final KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user"); + private Authorizer authorizer; private List<Action> actions = new ArrayList<>(); private RequestContext authorizeContext; private RequestContext authorizeByResourceTypeContext; @@ -94,6 +120,7 @@ public class AclAuthorizerBenchmark { @Setup(Level.Trial) public void setup() throws Exception { + authorizer = authorizerType.newAuthorizer(); prepareAclCache(); prepareAclToUpdate(); // By adding `-95` to the resource name prefix, we cause the `TreeMap.from/to` call to return @@ -177,9 +204,27 @@ public class AclAuthorizerBenchmark { } } + setupAcls(aclEntries); + } + + private void setupAcls(Map<ResourcePattern, Set<AclEntry>> aclEntries) { for (Map.Entry<ResourcePattern, Set<AclEntry>> entryMap : aclEntries.entrySet()) { - aclAuthorizer.updateCache(entryMap.getKey(), - new VersionedAcls(JavaConverters.asScalaSetConverter(entryMap.getValue()).asScala().toSet(), 1)); + ResourcePattern resourcePattern = entryMap.getKey(); + switch (authorizerType) { + case ACL: + ((AclAuthorizer) authorizer).updateCache(resourcePattern, + new VersionedAcls(JavaConverters.asScalaSetConverter(entryMap.getValue()).asScala().toSet(), 1)); + break; + case KRAFT: + + for (AclEntry aclEntry : entryMap.getValue()) { + StandardAcl standardAcl = StandardAcl.fromAclBinding(new AclBinding(resourcePattern, aclEntry)); + ((StandardAuthorizer) authorizer).addAcl(Uuid.randomUuid(), standardAcl); + + } + ((StandardAuthorizer) authorizer).completeInitialLoad(); + break; + } } } @@ -207,30 +252,31 @@ public class AclAuthorizerBenchmark { } @TearDown(Level.Trial) - public void tearDown() { - aclAuthorizer.close(); + public void tearDown() throws IOException { + authorizer.close(); } @Benchmark public void testAclsIterator() { - aclAuthorizer.acls(AclBindingFilter.ANY); + authorizer.acls(AclBindingFilter.ANY); } @Benchmark public void testAuthorizer() { - aclAuthorizer.authorize(authorizeContext, actions); + authorizer.authorize(authorizeContext, actions); } @Benchmark public void testAuthorizeByResourceType() { - aclAuthorizer.authorizeByResourceType(authorizeByResourceTypeContext, AclOperation.READ, ResourceType.TOPIC); + authorizer.authorizeByResourceType(authorizeByResourceTypeContext, AclOperation.READ, ResourceType.TOPIC); } @Benchmark public void testUpdateCache() { - AclAuthorizer aclAuthorizer = new AclAuthorizer(); - for (Map.Entry<ResourcePattern, VersionedAcls> e : aclToUpdate.entrySet()) { - aclAuthorizer.updateCache(e.getKey(), e.getValue()); + if (authorizerType == AuthorizerType.ACL) { + for (Map.Entry<ResourcePattern, VersionedAcls> e : aclToUpdate.entrySet()) { + ((AclAuthorizer) authorizer).updateCache(e.getKey(), e.getValue()); + } } } }