KAFKA-18888: Add KIP-877 support to Authorizer (#19050)

This also adds metrics to StandardAuthorizer

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>
This commit is contained in:
Mickael Maison 2025-04-15 19:40:24 +02:00 committed by GitHub
parent e9ca0bb0f6
commit fb2ce76b49
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
43 changed files with 663 additions and 248 deletions

View File

@ -176,6 +176,9 @@
<allow pkg="org.apache.kafka.controller" /> <allow pkg="org.apache.kafka.controller" />
<allow pkg="org.apache.kafka.metadata" /> <allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.common.internals" /> <allow pkg="org.apache.kafka.common.internals" />
<allow pkg="org.apache.kafka.common.metrics" />
<allow pkg="org.apache.kafka.common.metrics.internals" />
<allow pkg="org.apache.kafka.common.metrics.stats" />
</subpackage> </subpackage>
<subpackage name="bootstrap"> <subpackage name="bootstrap">
<allow pkg="org.apache.kafka.snapshot" /> <allow pkg="org.apache.kafka.snapshot" />

View File

@ -105,6 +105,7 @@
<subpackage name="security"> <subpackage name="security">
<allow pkg="org.apache.kafka.common.resource" /> <allow pkg="org.apache.kafka.common.resource" />
<allow pkg="org.apache.kafka.network" /> <allow pkg="org.apache.kafka.network" />
<allow pkg="org.apache.kafka.server" />
<allow pkg="org.apache.kafka.server.authorizer" /> <allow pkg="org.apache.kafka.server.authorizer" />
</subpackage> </subpackage>

View File

@ -30,6 +30,13 @@ import java.util.Optional;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier; import java.util.function.Supplier;
/**
* Plugins have the following tags:
* <ul>
* <li><code>config</code> set to the name of the configuration to specifying the plugin</li>
* <li><code>class</code> set to the name of the instance class</li>
* </ul>
*/
public class Plugin<T> implements Supplier<T>, AutoCloseable { public class Plugin<T> implements Supplier<T>, AutoCloseable {
private final T instance; private final T instance;
@ -40,14 +47,49 @@ public class Plugin<T> implements Supplier<T>, AutoCloseable {
this.pluginMetrics = Optional.ofNullable(pluginMetrics); this.pluginMetrics = Optional.ofNullable(pluginMetrics);
} }
/**
* Wrap an instance into a Plugin.
* @param instance the instance to wrap
* @param metrics the metrics
* @param tagsSupplier supplier to retrieve the tags
* @return the plugin
*/
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, Supplier<Map<String, String>> tagsSupplier) {
PluginMetricsImpl pluginMetrics = null;
if (instance instanceof Monitorable && metrics != null) {
pluginMetrics = new PluginMetricsImpl(metrics, tagsSupplier.get());
((Monitorable) instance).withPluginMetrics(pluginMetrics);
}
return new Plugin<>(instance, pluginMetrics);
}
/**
* Wrap an instance into a Plugin.
* @param instance the instance to wrap
* @param metrics the metrics
* @param key the value for the <code>config</code> tag
* @return the plugin
*/
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key) { public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key) {
return wrapInstance(instance, metrics, () -> tags(key, instance)); return wrapInstance(instance, metrics, () -> tags(key, instance));
} }
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key, Map<String, String> extraTags) { /**
* Wrap an instance into a Plugin.
* @param instance the instance to wrap
* @param metrics the metrics
* @param name extra tag name to add
* @param value extra tag value to add
* @param key the value for the <code>config</code> tag
* @return the plugin
*/
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key, String name, String value) {
Supplier<Map<String, String>> tagsSupplier = () -> {
Map<String, String> tags = tags(key, instance); Map<String, String> tags = tags(key, instance);
tags.putAll(extraTags); tags.put(name, value);
return wrapInstance(instance, metrics, () -> tags); return tags;
};
return wrapInstance(instance, metrics, tagsSupplier);
} }
private static <T> Map<String, String> tags(String key, T instance) { private static <T> Map<String, String> tags(String key, T instance) {
@ -57,6 +99,13 @@ public class Plugin<T> implements Supplier<T>, AutoCloseable {
return tags; return tags;
} }
/**
* Wrap a list of instances into Plugins.
* @param instances the instances to wrap
* @param metrics the metrics
* @param key the value for the <code>config</code> tag
* @return the list of plugins
*/
public static <T> List<Plugin<T>> wrapInstances(List<T> instances, Metrics metrics, String key) { public static <T> List<Plugin<T>> wrapInstances(List<T> instances, Metrics metrics, String key) {
List<Plugin<T>> plugins = new ArrayList<>(); List<Plugin<T>> plugins = new ArrayList<>();
for (T instance : instances) { for (T instance : instances) {
@ -65,15 +114,6 @@ public class Plugin<T> implements Supplier<T>, AutoCloseable {
return plugins; return plugins;
} }
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, Supplier<Map<String, String>> tagsSupplier) {
PluginMetricsImpl pluginMetrics = null;
if (instance instanceof Monitorable && metrics != null) {
pluginMetrics = new PluginMetricsImpl(metrics, tagsSupplier.get());
((Monitorable) instance).withPluginMetrics(pluginMetrics);
}
return new Plugin<>(instance, pluginMetrics);
}
@Override @Override
public T get() { public T get() {
return instance; return instance;

View File

@ -59,6 +59,10 @@ import java.util.concurrent.CompletionStage;
* *
* Authorizer implementation class may optionally implement @{@link org.apache.kafka.common.Reconfigurable} * Authorizer implementation class may optionally implement @{@link org.apache.kafka.common.Reconfigurable}
* to enable dynamic reconfiguration without restarting the broker. * to enable dynamic reconfiguration without restarting the broker.
* <p>Authorizer implementation class may also optionally implement {@link org.apache.kafka.common.metrics.Monitorable}
* to enable the authorizer to register metrics. The following tags are automatically added to all metrics registered:
* <code>config</code> set to <code>authorizer.class.name</code>, <code>class</code> set to the Authorizer class name,
* and <code>role</code> set to either <code>broker</code> or <code>controller</code>.
* <p> * <p>
* <b>Threading model:</b> * <b>Threading model:</b>
* <ul> * <ul>

View File

@ -27,7 +27,6 @@ import org.apache.kafka.server.config.ReplicationQuotaManagerConfig;
import org.apache.kafka.server.quota.ClientQuotaCallback; import org.apache.kafka.server.quota.ClientQuotaCallback;
import org.apache.kafka.server.quota.QuotaType; import org.apache.kafka.server.quota.QuotaType;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import scala.Option; import scala.Option;
@ -150,7 +149,7 @@ public class QuotaFactory {
clientQuotaCallback, clientQuotaCallback,
metrics, metrics,
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG,
Map.of("role", role) "role", role
)); ));
} }

View File

@ -28,6 +28,7 @@ import kafka.server.QuotaFactory.QuotaManagers;
import kafka.server.ReplicaManager; import kafka.server.ReplicaManager;
import kafka.server.share.SharePartitionManager; import kafka.server.share.SharePartitionManager;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.group.GroupConfigManager; import org.apache.kafka.coordinator.group.GroupConfigManager;
@ -58,7 +59,7 @@ public class KafkaApisBuilder {
private ConfigRepository configRepository = null; private ConfigRepository configRepository = null;
private MetadataCache metadataCache = null; private MetadataCache metadataCache = null;
private Metrics metrics = null; private Metrics metrics = null;
private Optional<Authorizer> authorizer = Optional.empty(); private Optional<Plugin<Authorizer>> authorizerPlugin = Optional.empty();
private QuotaManagers quotas = null; private QuotaManagers quotas = null;
private FetchManager fetchManager = null; private FetchManager fetchManager = null;
private SharePartitionManager sharePartitionManager = null; private SharePartitionManager sharePartitionManager = null;
@ -131,8 +132,8 @@ public class KafkaApisBuilder {
return this; return this;
} }
public KafkaApisBuilder setAuthorizer(Optional<Authorizer> authorizer) { public KafkaApisBuilder setAuthorizerPlugin(Optional<Plugin<Authorizer>> authorizerPlugin) {
this.authorizer = authorizer; this.authorizerPlugin = authorizerPlugin;
return this; return this;
} }
@ -219,7 +220,7 @@ public class KafkaApisBuilder {
configRepository, configRepository,
metadataCache, metadataCache,
metrics, metrics,
OptionConverters.toScala(authorizer), OptionConverters.toScala(authorizerPlugin),
quotas, quotas,
fetchManager, fetchManager,
sharePartitionManager, sharePartitionManager,

View File

@ -22,6 +22,7 @@ import kafka.utils.Logging
import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclBinding import org.apache.kafka.common.acl.AclBinding
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult
import org.apache.kafka.common.message._ import org.apache.kafka.common.message._
@ -46,7 +47,7 @@ import scala.jdk.OptionConverters.RichOptional
* Logic to handle ACL requests. * Logic to handle ACL requests.
*/ */
class AclApis(authHelper: AuthHelper, class AclApis(authHelper: AuthHelper,
authorizer: Option[Authorizer], authorizerPlugin: Option[Plugin[Authorizer]],
requestHelper: RequestHandlerHelper, requestHelper: RequestHandlerHelper,
role: ProcessRole, role: ProcessRole,
config: KafkaConfig) extends Logging { config: KafkaConfig) extends Logging {
@ -61,7 +62,7 @@ class AclApis(authHelper: AuthHelper,
def handleDescribeAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { def handleDescribeAcls(request: RequestChannel.Request): CompletableFuture[Unit] = {
authHelper.authorizeClusterOperation(request, DESCRIBE) authHelper.authorizeClusterOperation(request, DESCRIBE)
val describeAclsRequest = request.body[DescribeAclsRequest] val describeAclsRequest = request.body[DescribeAclsRequest]
authorizer match { authorizerPlugin match {
case None => case None =>
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeAclsResponse(new DescribeAclsResponseData() new DescribeAclsResponse(new DescribeAclsResponseData()
@ -74,7 +75,7 @@ class AclApis(authHelper: AuthHelper,
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeAclsResponse(new DescribeAclsResponseData() new DescribeAclsResponse(new DescribeAclsResponseData()
.setThrottleTimeMs(requestThrottleMs) .setThrottleTimeMs(requestThrottleMs)
.setResources(DescribeAclsResponse.aclsResources(auth.acls(filter))), .setResources(DescribeAclsResponse.aclsResources(auth.get.acls(filter))),
describeAclsRequest.version)) describeAclsRequest.version))
} }
CompletableFuture.completedFuture[Unit](()) CompletableFuture.completedFuture[Unit](())
@ -84,7 +85,7 @@ class AclApis(authHelper: AuthHelper,
authHelper.authorizeClusterOperation(request, ALTER) authHelper.authorizeClusterOperation(request, ALTER)
val createAclsRequest = request.body[CreateAclsRequest] val createAclsRequest = request.body[CreateAclsRequest]
authorizer match { authorizerPlugin match {
case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
createAclsRequest.getErrorResponse(requestThrottleMs, createAclsRequest.getErrorResponse(requestThrottleMs,
new SecurityDisabledException("No Authorizer is configured."))) new SecurityDisabledException("No Authorizer is configured.")))
@ -109,7 +110,7 @@ class AclApis(authHelper: AuthHelper,
} }
val future = new CompletableFuture[util.List[AclCreationResult]]() val future = new CompletableFuture[util.List[AclCreationResult]]()
val createResults = auth.createAcls(request.context, validBindings.asJava).stream().map(_.toCompletableFuture).toList val createResults = auth.get.createAcls(request.context, validBindings.asJava).stream().map(_.toCompletableFuture).toList
def sendResponseCallback(): Unit = { def sendResponseCallback(): Unit = {
val aclCreationResults = allBindings.map { acl => val aclCreationResults = allBindings.map { acl =>
@ -139,7 +140,7 @@ class AclApis(authHelper: AuthHelper,
def handleDeleteAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { def handleDeleteAcls(request: RequestChannel.Request): CompletableFuture[Unit] = {
authHelper.authorizeClusterOperation(request, ALTER) authHelper.authorizeClusterOperation(request, ALTER)
val deleteAclsRequest = request.body[DeleteAclsRequest] val deleteAclsRequest = request.body[DeleteAclsRequest]
authorizer match { authorizerPlugin match {
case None => case None =>
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
deleteAclsRequest.getErrorResponse(requestThrottleMs, deleteAclsRequest.getErrorResponse(requestThrottleMs,
@ -148,7 +149,7 @@ class AclApis(authHelper: AuthHelper,
case Some(auth) => case Some(auth) =>
val future = new CompletableFuture[util.List[DeleteAclsFilterResult]]() val future = new CompletableFuture[util.List[DeleteAclsFilterResult]]()
val deleteResults = auth.deleteAcls(request.context, deleteAclsRequest.filters) val deleteResults = auth.get.deleteAcls(request.context, deleteAclsRequest.filters)
.stream().map(_.toCompletableFuture).toList .stream().map(_.toCompletableFuture).toList
def sendResponseCallback(): Unit = { def sendResponseCallback(): Unit = {

View File

@ -24,6 +24,7 @@ import org.apache.kafka.clients.admin.EndpointType
import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.acl.AclOperation.DESCRIBE import org.apache.kafka.common.acl.AclOperation.DESCRIBE
import org.apache.kafka.common.errors.ClusterAuthorizationException import org.apache.kafka.common.errors.ClusterAuthorizationException
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.message.DescribeClusterResponseData import org.apache.kafka.common.message.DescribeClusterResponseData
import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
@ -38,7 +39,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authoriz
import scala.collection.Seq import scala.collection.Seq
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
class AuthHelper(authorizer: Option[Authorizer]) { class AuthHelper(authorizer: Option[Plugin[Authorizer]]) {
def authorize(requestContext: RequestContext, def authorize(requestContext: RequestContext,
operation: AclOperation, operation: AclOperation,
resourceType: ResourceType, resourceType: ResourceType,
@ -49,7 +50,7 @@ class AuthHelper(authorizer: Option[Authorizer]) {
authorizer.forall { authZ => authorizer.forall { authZ =>
val resource = new ResourcePattern(resourceType, resourceName, PatternType.LITERAL) val resource = new ResourcePattern(resourceType, resourceName, PatternType.LITERAL)
val actions = Collections.singletonList(new Action(operation, resource, refCount, logIfAllowed, logIfDenied)) val actions = Collections.singletonList(new Action(operation, resource, refCount, logIfAllowed, logIfDenied))
authZ.authorize(requestContext, actions).get(0) == AuthorizationResult.ALLOWED authZ.get.authorize(requestContext, actions).get(0) == AuthorizationResult.ALLOWED
} }
} }
@ -64,7 +65,7 @@ class AuthHelper(authorizer: Option[Authorizer]) {
case Some(authZ) => case Some(authZ) =>
val resourcePattern = new ResourcePattern(resource.resourceType, resource.name, PatternType.LITERAL) val resourcePattern = new ResourcePattern(resource.resourceType, resource.name, PatternType.LITERAL)
val actions = supportedOps.map { op => new Action(op, resourcePattern, 1, false, false) } val actions = supportedOps.map { op => new Action(op, resourcePattern, 1, false, false) }
authZ.authorize(request.context, actions.asJava).asScala authZ.get.authorize(request.context, actions.asJava).asScala
.zip(supportedOps) .zip(supportedOps)
.filter(_._1 == AuthorizationResult.ALLOWED) .filter(_._1 == AuthorizationResult.ALLOWED)
.map(_._2).toSet .map(_._2).toSet
@ -77,7 +78,7 @@ class AuthHelper(authorizer: Option[Authorizer]) {
def authorizeByResourceType(requestContext: RequestContext, operation: AclOperation, def authorizeByResourceType(requestContext: RequestContext, operation: AclOperation,
resourceType: ResourceType): Boolean = { resourceType: ResourceType): Boolean = {
authorizer.forall { authZ => authorizer.forall { authZ =>
authZ.authorizeByResourceType(requestContext, operation, resourceType) == AuthorizationResult.ALLOWED authZ.get.authorizeByResourceType(requestContext, operation, resourceType) == AuthorizationResult.ALLOWED
} }
} }
@ -109,7 +110,7 @@ class AuthHelper(authorizer: Option[Authorizer]) {
val resource = new ResourcePattern(resourceType, resourceName, PatternType.LITERAL) val resource = new ResourcePattern(resourceType, resourceName, PatternType.LITERAL)
new Action(operation, resource, count, logIfAllowed, logIfDenied) new Action(operation, resource, count, logIfAllowed, logIfDenied)
}.toBuffer }.toBuffer
authZ.authorize(requestContext, actions.asJava).asScala authZ.get.authorize(requestContext, actions.asJava).asScala
.zip(resourceNameToCount.keySet) .zip(resourceNameToCount.keySet)
.collect { case (authzResult, resourceName) if authzResult == AuthorizationResult.ALLOWED => .collect { case (authzResult, resourceName) if authzResult == AuthorizationResult.ALLOWED =>
resourceName resourceName

View File

@ -27,6 +27,7 @@ import kafka.server.metadata._
import kafka.server.share.{ShareCoordinatorMetadataCacheHelperImpl, SharePartitionManager} import kafka.server.share.{ShareCoordinatorMetadataCacheHelperImpl, SharePartitionManager}
import kafka.utils.CoreUtils import kafka.utils.CoreUtils
import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
@ -103,7 +104,7 @@ class BrokerServer(
@volatile var dataPlaneRequestProcessor: KafkaApis = _ @volatile var dataPlaneRequestProcessor: KafkaApis = _
var authorizer: Option[Authorizer] = None var authorizerPlugin: Option[Plugin[Authorizer]] = None
@volatile var socketServer: SocketServer = _ @volatile var socketServer: SocketServer = _
var dataPlaneRequestHandlerPool: KafkaRequestHandlerPool = _ var dataPlaneRequestHandlerPool: KafkaRequestHandlerPool = _
@ -412,8 +413,7 @@ class BrokerServer(
) )
// Create and initialize an authorizer if one is configured. // Create and initialize an authorizer if one is configured.
authorizer = config.createNewAuthorizer() authorizerPlugin = config.createNewAuthorizer(metrics, ProcessRole.BrokerRole.toString)
authorizer.foreach(_.configure(config.originals))
// The FetchSessionCache is divided into config.numIoThreads shards, each responsible // The FetchSessionCache is divided into config.numIoThreads shards, each responsible
// for Math.max(1, shardNum * sessionIdRange) <= sessionId < (shardNum + 1) * sessionIdRange // for Math.max(1, shardNum * sessionIdRange) <= sessionId < (shardNum + 1) * sessionIdRange
@ -456,7 +456,7 @@ class BrokerServer(
configRepository = metadataCache, configRepository = metadataCache,
metadataCache = metadataCache, metadataCache = metadataCache,
metrics = metrics, metrics = metrics,
authorizer = authorizer, authorizerPlugin = authorizerPlugin,
quotas = quotaManagers, quotas = quotaManagers,
fetchManager = fetchManager, fetchManager = fetchManager,
sharePartitionManager = sharePartitionManager, sharePartitionManager = sharePartitionManager,
@ -529,7 +529,7 @@ class BrokerServer(
config.nodeId, config.nodeId,
sharedServer.metadataPublishingFaultHandler, sharedServer.metadataPublishingFaultHandler,
"broker", "broker",
authorizer.toJava authorizerPlugin.toJava
), ),
sharedServer.initialBrokerMetadataLoadFaultHandler, sharedServer.initialBrokerMetadataLoadFaultHandler,
sharedServer.metadataPublishingFaultHandler sharedServer.metadataPublishingFaultHandler
@ -586,7 +586,7 @@ class BrokerServer(
// authorizer future is completed. // authorizer future is completed.
val endpointReadyFutures = { val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder() val builder = new EndpointReadyFutures.Builder()
builder.build(authorizer.toJava, builder.build(authorizerPlugin.toJava,
new KafkaAuthorizerServerInfo( new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId), new ClusterResource(clusterId),
config.nodeId, config.nodeId,
@ -645,7 +645,7 @@ class BrokerServer(
.withGroupCoordinatorMetrics(new GroupCoordinatorMetrics(KafkaYammerMetrics.defaultRegistry, metrics)) .withGroupCoordinatorMetrics(new GroupCoordinatorMetrics(KafkaYammerMetrics.defaultRegistry, metrics))
.withGroupConfigManager(groupConfigManager) .withGroupConfigManager(groupConfigManager)
.withPersister(persister) .withPersister(persister)
.withAuthorizer(authorizer.toJava) .withAuthorizerPlugin(authorizerPlugin.toJava)
.build() .build()
} }
@ -765,7 +765,7 @@ class BrokerServer(
CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this) CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this)
if (dataPlaneRequestProcessor != null) if (dataPlaneRequestProcessor != null)
CoreUtils.swallow(dataPlaneRequestProcessor.close(), this) CoreUtils.swallow(dataPlaneRequestProcessor.close(), this)
authorizer.foreach(Utils.closeQuietly(_, "authorizer")) authorizerPlugin.foreach(Utils.closeQuietly(_, "authorizer plugin"))
/** /**
* We must shutdown the scheduler early because otherwise, the scheduler could touch other * We must shutdown the scheduler early because otherwise, the scheduler could touch other

View File

@ -34,8 +34,7 @@ import org.apache.kafka.common.Uuid.ZERO_UUID
import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, CREATE_TOKENS, DELETE, DESCRIBE, DESCRIBE_CONFIGS} import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, CREATE_TOKENS, DELETE, DESCRIBE, DESCRIBE_CONFIGS}
import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException, UnsupportedVersionException} import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException, UnsupportedVersionException}
import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.internals.{FatalExitError, Plugin, Topic}
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse} import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse}
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult
@ -68,7 +67,7 @@ import scala.jdk.CollectionConverters._
*/ */
class ControllerApis( class ControllerApis(
val requestChannel: RequestChannel, val requestChannel: RequestChannel,
val authorizer: Option[Authorizer], val authorizerPlugin: Option[Plugin[Authorizer]],
val quotas: QuotaManagers, val quotas: QuotaManagers,
val time: Time, val time: Time,
val controller: Controller, val controller: Controller,
@ -81,11 +80,11 @@ class ControllerApis(
) extends ApiRequestHandler with Logging { ) extends ApiRequestHandler with Logging {
this.logIdent = s"[ControllerApis nodeId=${config.nodeId}] " this.logIdent = s"[ControllerApis nodeId=${config.nodeId}] "
val authHelper = new AuthHelper(authorizer) val authHelper = new AuthHelper(authorizerPlugin)
val configHelper = new ConfigHelper(metadataCache, config, metadataCache) val configHelper = new ConfigHelper(metadataCache, config, metadataCache)
val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time) val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time)
val runtimeLoggerManager = new RuntimeLoggerManager(config.nodeId, logger.underlying) val runtimeLoggerManager = new RuntimeLoggerManager(config.nodeId, logger.underlying)
private val aclApis = new AclApis(authHelper, authorizer, requestHelper, ProcessRole.ControllerRole, config) private val aclApis = new AclApis(authHelper, authorizerPlugin, requestHelper, ProcessRole.ControllerRole, config)
def isClosed: Boolean = aclApis.isClosed def isClosed: Boolean = aclApis.isClosed

View File

@ -24,6 +24,7 @@ import kafka.server.QuotaFactory.QuotaManagers
import scala.collection.immutable import scala.collection.immutable
import kafka.server.metadata.{ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} import kafka.server.metadata.{ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher}
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
@ -82,7 +83,7 @@ class ControllerServer(
var status: ProcessStatus = SHUTDOWN var status: ProcessStatus = SHUTDOWN
var linuxIoMetricsCollector: LinuxIoMetricsCollector = _ var linuxIoMetricsCollector: LinuxIoMetricsCollector = _
@volatile var authorizer: Option[Authorizer] = None @volatile var authorizerPlugin: Option[Plugin[Authorizer]] = None
var tokenCache: DelegationTokenCache = _ var tokenCache: DelegationTokenCache = _
var credentialProvider: CredentialProvider = _ var credentialProvider: CredentialProvider = _
var socketServer: SocketServer = _ var socketServer: SocketServer = _
@ -138,8 +139,7 @@ class ControllerServer(
metricsGroup.newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes()) metricsGroup.newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes())
} }
authorizer = config.createNewAuthorizer() authorizerPlugin = config.createNewAuthorizer(metrics, ProcessRole.ControllerRole.toString)
authorizer.foreach(_.configure(config.originals))
metadataCache = new KRaftMetadataCache(config.nodeId, () => raftManager.client.kraftVersion()) metadataCache = new KRaftMetadataCache(config.nodeId, () => raftManager.client.kraftVersion())
@ -181,7 +181,7 @@ class ControllerServer(
val endpointReadyFutures = { val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder() val builder = new EndpointReadyFutures.Builder()
builder.build(authorizer.toJava, builder.build(authorizerPlugin.toJava,
new KafkaAuthorizerServerInfo( new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId), new ClusterResource(clusterId),
config.nodeId, config.nodeId,
@ -260,10 +260,12 @@ class ControllerServer(
// If we are using a ClusterMetadataAuthorizer, requests to add or remove ACLs must go // If we are using a ClusterMetadataAuthorizer, requests to add or remove ACLs must go
// through the controller. // through the controller.
authorizer match { authorizerPlugin.foreach { plugin =>
case Some(a: ClusterMetadataAuthorizer) => a.setAclMutator(controller) plugin.get match {
case a: ClusterMetadataAuthorizer => a.setAclMutator(controller)
case _ => case _ =>
} }
}
quotaManagers = QuotaFactory.instantiate(config, quotaManagers = QuotaFactory.instantiate(config,
metrics, metrics,
@ -271,7 +273,7 @@ class ControllerServer(
s"controller-${config.nodeId}-", ProcessRole.ControllerRole.toString) s"controller-${config.nodeId}-", ProcessRole.ControllerRole.toString)
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas) clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel, controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel,
authorizer, authorizerPlugin,
quotaManagers, quotaManagers,
time, time,
controller, controller,
@ -376,7 +378,7 @@ class ControllerServer(
config.nodeId, config.nodeId,
sharedServer.metadataPublishingFaultHandler, sharedServer.metadataPublishingFaultHandler,
"controller", "controller",
authorizer.toJava authorizerPlugin.toJava
)) ))
// Install all metadata publishers. // Install all metadata publishers.
@ -468,7 +470,7 @@ class ControllerServer(
CoreUtils.swallow(quotaManagers.shutdown(), this) CoreUtils.swallow(quotaManagers.shutdown(), this)
Utils.closeQuietly(controller, "controller") Utils.closeQuietly(controller, "controller")
Utils.closeQuietly(quorumControllerMetrics, "quorum controller metrics") Utils.closeQuietly(quorumControllerMetrics, "quorum controller metrics")
authorizer.foreach(Utils.closeQuietly(_, "authorizer")) authorizerPlugin.foreach(Utils.closeQuietly(_, "authorizer plugin"))
createTopicPolicy.foreach(policy => Utils.closeQuietly(policy, "create topic policy")) createTopicPolicy.foreach(policy => Utils.closeQuietly(policy, "create topic policy"))
alterConfigPolicy.foreach(policy => Utils.closeQuietly(policy, "alter config policy")) alterConfigPolicy.foreach(policy => Utils.closeQuietly(policy, "alter config policy"))
socketServerFirstBoundPortFuture.completeExceptionally(new RuntimeException("shutting down")) socketServerFirstBoundPortFuture.completeExceptionally(new RuntimeException("shutting down"))

View File

@ -283,10 +283,12 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
* directly. They are provided both old and new configs. * directly. They are provided both old and new configs.
*/ */
def addReconfigurables(kafkaServer: KafkaBroker): Unit = { def addReconfigurables(kafkaServer: KafkaBroker): Unit = {
kafkaServer.authorizer match { kafkaServer.authorizerPlugin.foreach { plugin =>
case Some(authz: Reconfigurable) => addReconfigurable(authz) plugin.get match {
case authz: Reconfigurable => addReconfigurable(authz)
case _ => case _ =>
} }
}
addReconfigurable(kafkaServer.kafkaYammerMetrics) addReconfigurable(kafkaServer.kafkaYammerMetrics)
addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId)) addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId))
addReconfigurable(new DynamicClientQuotaCallback(kafkaServer.quotaManagers, kafkaServer.config)) addReconfigurable(new DynamicClientQuotaCallback(kafkaServer.quotaManagers, kafkaServer.config))
@ -303,10 +305,12 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
* Add reconfigurables to be notified when a dynamic controller config is updated. * Add reconfigurables to be notified when a dynamic controller config is updated.
*/ */
def addReconfigurables(controller: ControllerServer): Unit = { def addReconfigurables(controller: ControllerServer): Unit = {
controller.authorizer match { controller.authorizerPlugin.foreach { plugin =>
case Some(authz: Reconfigurable) => addReconfigurable(authz) plugin.get match {
case authz: Reconfigurable => addReconfigurable(authz)
case _ => case _ =>
} }
}
if (!kafkaConfig.processRoles.contains(ProcessRole.BrokerRole)) { if (!kafkaConfig.processRoles.contains(ProcessRole.BrokerRole)) {
// only add these if the controller isn't also running the broker role // only add these if the controller isn't also running the broker role
// because these would already be added via the broker in that case // because these would already be added via the broker in that case

View File

@ -29,7 +29,7 @@ import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal}
import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.internals.{FatalExitError, Plugin, Topic}
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.{AddPartitionsToTxnResult, AddPartitionsToTxnResultCollection} import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.{AddPartitionsToTxnResult, AddPartitionsToTxnResultCollection}
import org.apache.kafka.common.message.DeleteRecordsResponseData.{DeleteRecordsPartitionResult, DeleteRecordsTopicResult} import org.apache.kafka.common.message.DeleteRecordsResponseData.{DeleteRecordsPartitionResult, DeleteRecordsTopicResult}
import org.apache.kafka.common.message.DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic import org.apache.kafka.common.message.DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic
@ -99,7 +99,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val configRepository: ConfigRepository, val configRepository: ConfigRepository,
val metadataCache: MetadataCache, val metadataCache: MetadataCache,
val metrics: Metrics, val metrics: Metrics,
val authorizer: Option[Authorizer], val authorizerPlugin: Option[Plugin[Authorizer]],
val quotas: QuotaManagers, val quotas: QuotaManagers,
val fetchManager: FetchManager, val fetchManager: FetchManager,
val sharePartitionManager: SharePartitionManager, val sharePartitionManager: SharePartitionManager,
@ -115,9 +115,9 @@ class KafkaApis(val requestChannel: RequestChannel,
type FetchResponseStats = Map[TopicPartition, RecordValidationStats] type FetchResponseStats = Map[TopicPartition, RecordValidationStats]
this.logIdent = "[KafkaApi-%d] ".format(brokerId) this.logIdent = "[KafkaApi-%d] ".format(brokerId)
val configHelper = new ConfigHelper(metadataCache, config, configRepository) val configHelper = new ConfigHelper(metadataCache, config, configRepository)
val authHelper = new AuthHelper(authorizer) val authHelper = new AuthHelper(authorizerPlugin)
val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time) val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time)
val aclApis = new AclApis(authHelper, authorizer, requestHelper, ProcessRole.BrokerRole, config) val aclApis = new AclApis(authHelper, authorizerPlugin, requestHelper, ProcessRole.BrokerRole, config)
val configManager = new ConfigAdminManager(brokerId, config, configRepository) val configManager = new ConfigAdminManager(brokerId, config, configRepository)
val describeTopicPartitionsRequestHandler = new DescribeTopicPartitionsRequestHandler( val describeTopicPartitionsRequestHandler = new DescribeTopicPartitionsRequestHandler(
metadataCache, authHelper, config) metadataCache, authHelper, config)
@ -2598,7 +2598,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
// Clients are not allowed to see topics that are not authorized for Describe. // Clients are not allowed to see topics that are not authorized for Describe.
if (authorizer.isDefined) { if (authorizerPlugin.isDefined) {
val topicsToCheck = response.groups.stream() val topicsToCheck = response.groups.stream()
.flatMap(group => group.members.stream) .flatMap(group => group.members.stream)
.flatMap(member => util.stream.Stream.of(member.assignment, member.targetAssignment)) .flatMap(member => util.stream.Stream.of(member.assignment, member.targetAssignment))
@ -2779,7 +2779,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
// Clients are not allowed to see topics that are not authorized for Describe. // Clients are not allowed to see topics that are not authorized for Describe.
if (authorizer.isDefined) { if (authorizerPlugin.isDefined) {
val topicsToCheck = response.groups.stream() val topicsToCheck = response.groups.stream()
.filter(group => group.topology != null) .filter(group => group.topology != null)
.flatMap(group => group.topology.subtopologies.stream) .flatMap(group => group.topology.subtopologies.stream)
@ -2950,7 +2950,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
// Clients are not allowed to see topics that are not authorized for Describe. // Clients are not allowed to see topics that are not authorized for Describe.
if (authorizer.isDefined) { if (authorizerPlugin.isDefined) {
val topicsToCheck = response.groups.stream() val topicsToCheck = response.groups.stream()
.flatMap(group => group.members.stream) .flatMap(group => group.members.stream)
.flatMap(member => member.assignment.topicPartitions.stream) .flatMap(member => member.assignment.topicPartitions.stream)

View File

@ -22,7 +22,7 @@ import kafka.log.remote.RemoteLogManager
import kafka.network.SocketServer import kafka.network.SocketServer
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.common.ClusterResource import org.apache.kafka.common.ClusterResource
import org.apache.kafka.common.internals.ClusterResourceListeners import org.apache.kafka.common.internals.{ClusterResourceListeners, Plugin}
import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} import org.apache.kafka.common.metrics.{Metrics, MetricsReporter}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
@ -78,7 +78,7 @@ trait KafkaBroker extends Logging {
// acquire lock while handling Fetch requests. // acquire lock while handling Fetch requests.
val NumFetchSessionCacheShards: Int = 8 val NumFetchSessionCacheShards: Int = 8
def authorizer: Option[Authorizer] def authorizerPlugin: Option[Plugin[Authorizer]]
def brokerState: BrokerState def brokerState: BrokerState
def clusterId: String def clusterId: String
def config: KafkaConfig def config: KafkaConfig

View File

@ -27,6 +27,8 @@ import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResourc
import org.apache.kafka.common.config.ConfigDef.ConfigKey import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.config.types.Password
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.security.auth.KafkaPrincipalSerde import org.apache.kafka.common.security.auth.KafkaPrincipalSerde
@ -270,12 +272,12 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
} }
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
def createNewAuthorizer(): Option[Authorizer] = { def createNewAuthorizer(metrics: Metrics, role: String): Option[Plugin[Authorizer]] = {
val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG) val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)
if (className == null || className.isEmpty) if (className == null || className.isEmpty)
None None
else { else {
Some(AuthorizerUtils.createAuthorizer(className)) Some(AuthorizerUtils.createAuthorizer(className, originals, metrics, ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, role))
} }
} }

View File

@ -53,6 +53,7 @@ import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition} import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition}
import org.apache.kafka.server.log.remote.TopicPartitionLog import org.apache.kafka.server.log.remote.TopicPartitionLog
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey} import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey}
@ -2660,7 +2661,7 @@ class ReplicaManager(val config: KafkaConfig,
config.replicaSelectorClassName.map { className => config.replicaSelectorClassName.map { className =>
val tmpReplicaSelector: ReplicaSelector = Utils.newInstance(className, classOf[ReplicaSelector]) val tmpReplicaSelector: ReplicaSelector = Utils.newInstance(className, classOf[ReplicaSelector])
tmpReplicaSelector.configure(config.originals()) tmpReplicaSelector.configure(config.originals())
Plugin.wrapInstance(tmpReplicaSelector, metrics, className) Plugin.wrapInstance(tmpReplicaSelector, metrics, ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG)
} }
} }

View File

@ -27,6 +27,7 @@ import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.memory.MemoryPool;
import org.apache.kafka.common.message.DescribeTopicPartitionsRequestData; import org.apache.kafka.common.message.DescribeTopicPartitionsRequestData;
import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData; import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData;
@ -113,6 +114,7 @@ class DescribeTopicPartitionsRequestHandlerTest {
void testDescribeTopicPartitionsRequest() { void testDescribeTopicPartitionsRequest() {
// 1. Set up authorizer // 1. Set up authorizer
Authorizer authorizer = mock(Authorizer.class); Authorizer authorizer = mock(Authorizer.class);
Plugin<Authorizer> authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name");
String unauthorizedTopic = "unauthorized-topic"; String unauthorizedTopic = "unauthorized-topic";
String authorizedTopic = "authorized-topic"; String authorizedTopic = "authorized-topic";
String authorizedNonExistTopic = "authorized-non-exist"; String authorizedNonExistTopic = "authorized-non-exist";
@ -190,7 +192,7 @@ class DescribeTopicPartitionsRequestHandlerTest {
KRaftMetadataCache metadataCache = new KRaftMetadataCache(0, () -> KRaftVersion.KRAFT_VERSION_1); KRaftMetadataCache metadataCache = new KRaftMetadataCache(0, () -> KRaftVersion.KRAFT_VERSION_1);
updateKraftMetadataCache(metadataCache, records); updateKraftMetadataCache(metadataCache, records);
DescribeTopicPartitionsRequestHandler handler = DescribeTopicPartitionsRequestHandler handler =
new DescribeTopicPartitionsRequestHandler(metadataCache, new AuthHelper(scala.Option.apply(authorizer)), createKafkaDefaultConfig()); new DescribeTopicPartitionsRequestHandler(metadataCache, new AuthHelper(scala.Option.apply(authorizerPlugin)), createKafkaDefaultConfig());
// 3.1 Basic test // 3.1 Basic test
DescribeTopicPartitionsRequest describeTopicPartitionsRequest = new DescribeTopicPartitionsRequest( DescribeTopicPartitionsRequest describeTopicPartitionsRequest = new DescribeTopicPartitionsRequest(
@ -312,6 +314,7 @@ class DescribeTopicPartitionsRequestHandlerTest {
void testDescribeTopicPartitionsRequestWithEdgeCases() { void testDescribeTopicPartitionsRequestWithEdgeCases() {
// 1. Set up authorizer // 1. Set up authorizer
Authorizer authorizer = mock(Authorizer.class); Authorizer authorizer = mock(Authorizer.class);
Plugin<Authorizer> authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name");
String authorizedTopic = "authorized-topic1"; String authorizedTopic = "authorized-topic1";
String authorizedTopic2 = "authorized-topic2"; String authorizedTopic2 = "authorized-topic2";
@ -387,7 +390,7 @@ class DescribeTopicPartitionsRequestHandlerTest {
KRaftMetadataCache metadataCache = new KRaftMetadataCache(0, () -> KRaftVersion.KRAFT_VERSION_1); KRaftMetadataCache metadataCache = new KRaftMetadataCache(0, () -> KRaftVersion.KRAFT_VERSION_1);
updateKraftMetadataCache(metadataCache, records); updateKraftMetadataCache(metadataCache, records);
DescribeTopicPartitionsRequestHandler handler = DescribeTopicPartitionsRequestHandler handler =
new DescribeTopicPartitionsRequestHandler(metadataCache, new AuthHelper(scala.Option.apply(authorizer)), createKafkaDefaultConfig()); new DescribeTopicPartitionsRequestHandler(metadataCache, new AuthHelper(scala.Option.apply(authorizerPlugin)), createKafkaDefaultConfig());
// 3.1 With cursor point to the first one // 3.1 With cursor point to the first one
DescribeTopicPartitionsRequest describeTopicPartitionsRequest = new DescribeTopicPartitionsRequest(new DescribeTopicPartitionsRequestData() DescribeTopicPartitionsRequest describeTopicPartitionsRequest = new DescribeTopicPartitionsRequest(new DescribeTopicPartitionsRequestData()

View File

@ -2641,8 +2641,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
map(_.toCompletableFuture.get).flatMap { deletion => map(_.toCompletableFuture.get).flatMap { deletion =>
deletion.aclBindingDeleteResults().asScala.map(_.aclBinding.pattern).toSet deletion.aclBindingDeleteResults().asScala.map(_.aclBinding.pattern).toSet
}.foreach { resource => }.foreach { resource =>
(brokers.map(_.authorizer.get) ++ controllerServers.map(_.authorizer.get)).foreach { authorizer => (brokers.map(_.authorizerPlugin.get) ++ controllerServers.map(_.authorizerPlugin.get)).foreach { authorizer =>
TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], authorizer, resource, aclEntryFilter) TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], authorizer.get, resource, aclEntryFilter)
} }
} }
} }

View File

@ -70,7 +70,7 @@ class DelegationTokenEndToEndAuthorizationWithOwnerTest extends DelegationTokenE
superuserAdminClient.createAcls(List(AclTokenOtherDescribe, AclTokenCreate, AclTokenDescribe).asJava).values superuserAdminClient.createAcls(List(AclTokenOtherDescribe, AclTokenCreate, AclTokenDescribe).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TokenCreateAcl ++ TokenDescribeAcl, s.dataPlaneRequestProcessor.authorizer.get, TestUtils.waitAndVerifyAcls(TokenCreateAcl ++ TokenDescribeAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get,
new ResourcePattern(USER, clientPrincipal.toString, LITERAL)) new ResourcePattern(USER, clientPrincipal.toString, LITERAL))
} }
} }

View File

@ -251,8 +251,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclWildcardGroupRead).asJava).values superuserAdminClient.createAcls(List(AclWildcardGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, wildcardTopicResource) TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, wildcardTopicResource)
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, wildcardGroupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, wildcardGroupResource)
} }
} }
@ -262,8 +262,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclPrefixedGroupRead).asJava).values superuserAdminClient.createAcls(List(AclPrefixedGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, prefixedTopicResource) TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, prefixedTopicResource)
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, prefixedGroupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, prefixedGroupResource)
} }
} }
@ -276,9 +276,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclGroupRead).asJava).values superuserAdminClient.createAcls(List(AclGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, TestUtils.waitAndVerifyAcls(TopicReadAcl ++ TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get,
new ResourcePattern(TOPIC, tp.topic, LITERAL)) new ResourcePattern(TOPIC, tp.topic, LITERAL))
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, groupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, groupResource)
} }
} }
@ -292,7 +292,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
val superuserAdminClient = createSuperuserAdminClient() val superuserAdminClient = createSuperuserAdminClient()
superuserAdminClient.createAcls(List(AclGroupRead).asJava).values superuserAdminClient.createAcls(List(AclGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, groupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, groupResource)
} }
} }
@ -382,7 +382,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclTopicDescribe()).asJava).values superuserAdminClient.createAcls(List(AclTopicDescribe()).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicDescribeAcl, s.dataPlaneRequestProcessor.authorizer.get, topicResource) TestUtils.waitAndVerifyAcls(TopicDescribeAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, topicResource)
} }
val prop = new Properties() val prop = new Properties()
@ -450,8 +450,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclGroupRead).asJava).values superuserAdminClient.createAcls(List(AclGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, topicResource) TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, topicResource)
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, groupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, groupResource)
} }
val producer = createProducer() val producer = createProducer()
@ -461,8 +461,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.deleteAcls(List(AclTopicWrite().toFilter).asJava).values superuserAdminClient.deleteAcls(List(AclTopicWrite().toFilter).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, topicResource) TestUtils.waitAndVerifyAcls(TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, topicResource)
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, groupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, groupResource)
} }
} }
@ -496,8 +496,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
superuserAdminClient.createAcls(List(AclGroupRead).asJava).values superuserAdminClient.createAcls(List(AclGroupRead).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, topicResource) TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, topicResource)
TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizer.get, groupResource) TestUtils.waitAndVerifyAcls(GroupReadAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, groupResource)
} }
val producer = createProducer() val producer = createProducer()
sendRecords(producer, numRecords, tp) sendRecords(producer, numRecords, tp)
@ -513,7 +513,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
val superuserAdminClient = createSuperuserAdminClient() val superuserAdminClient = createSuperuserAdminClient()
superuserAdminClient.createAcls(List(AclTopicWrite(), AclTopicCreate(), AclTopicDescribe()).asJava).values superuserAdminClient.createAcls(List(AclTopicWrite(), AclTopicCreate(), AclTopicDescribe()).asJava).values
brokers.foreach { s => brokers.foreach { s =>
TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizer.get, topicResource) TestUtils.waitAndVerifyAcls(TopicWriteAcl ++ TopicDescribeAcl ++ TopicCreateAcl, s.dataPlaneRequestProcessor.authorizerPlugin.get, topicResource)
} }
val producer = createProducer() val producer = createProducer()
sendRecords(producer, numRecords, tp) sendRecords(producer, numRecords, tp)

View File

@ -23,6 +23,7 @@ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUS
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.config.{ConfigResource, SaslConfigs, TopicConfig} import org.apache.kafka.common.config.{ConfigResource, SaslConfigs, TopicConfig}
import org.apache.kafka.common.errors.{ClusterAuthorizationException, DelegationTokenExpiredException, DelegationTokenNotFoundException, InvalidRequestException, TimeoutException, TopicAuthorizationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.errors.{ClusterAuthorizationException, DelegationTokenExpiredException, DelegationTokenNotFoundException, InvalidRequestException, TimeoutException, TopicAuthorizationException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.PatternType.LITERAL
import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC} import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC}
import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType}
@ -96,9 +97,9 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
superUserAdmin.createAcls(clusterAcls.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava) superUserAdmin.createAcls(clusterAcls.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava)
brokers.foreach { b => brokers.foreach { b =>
TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(TOPIC, "*", LITERAL)) TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizerPlugin.get, new ResourcePattern(TOPIC, "*", LITERAL))
TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(GROUP, "*", LITERAL)) TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizerPlugin.get, new ResourcePattern(GROUP, "*", LITERAL))
TestUtils.waitAndVerifyAcls(clusterAcls.toSet, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) TestUtils.waitAndVerifyAcls(clusterAcls.toSet, b.dataPlaneRequestProcessor.authorizerPlugin.get, clusterResourcePattern)
} }
} }
@ -283,7 +284,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete only ACLs on literal 'mytopic2' topic // Delete only ACLs on literal 'mytopic2' topic
var deleted = client.deleteAcls(List(acl2.toFilter).asJava).all().get().asScala.toSet var deleted = client.deleteAcls(List(acl2.toFilter).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(acl2.entry(), b.dataPlaneRequestProcessor.authorizer.get, acl2.pattern()) waitAndVerifyRemovedAcl(acl2.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, acl2.pattern())
} }
assertEquals(Set(anyAcl, fooAcl, prefixAcl), getAcls(allTopicAcls)) assertEquals(Set(anyAcl, fooAcl, prefixAcl), getAcls(allTopicAcls))
@ -292,7 +293,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete only ACLs on literal '*' topic // Delete only ACLs on literal '*' topic
deleted = client.deleteAcls(List(anyAcl.toFilter).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(anyAcl.toFilter).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(anyAcl.entry(), b.dataPlaneRequestProcessor.authorizer.get, anyAcl.pattern()) waitAndVerifyRemovedAcl(anyAcl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, anyAcl.pattern())
} }
assertEquals(Set(acl2, fooAcl, prefixAcl), getAcls(allTopicAcls)) assertEquals(Set(acl2, fooAcl, prefixAcl), getAcls(allTopicAcls))
@ -301,7 +302,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete only ACLs on specific prefixed 'mytopic' topics: // Delete only ACLs on specific prefixed 'mytopic' topics:
deleted = client.deleteAcls(List(prefixAcl.toFilter).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(prefixAcl.toFilter).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(prefixAcl.entry(), b.dataPlaneRequestProcessor.authorizer.get, prefixAcl.pattern()) waitAndVerifyRemovedAcl(prefixAcl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, prefixAcl.pattern())
} }
assertEquals(Set(anyAcl, acl2, fooAcl), getAcls(allTopicAcls)) assertEquals(Set(anyAcl, acl2, fooAcl), getAcls(allTopicAcls))
@ -311,7 +312,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
deleted = client.deleteAcls(List(allLiteralTopicAcls).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(allLiteralTopicAcls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
Set(anyAcl, acl2, fooAcl).foreach(acl => Set(anyAcl, acl2, fooAcl).foreach(acl =>
waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizer.get, acl.pattern()) waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, acl.pattern())
) )
} }
assertEquals(Set(prefixAcl), getAcls(allTopicAcls)) assertEquals(Set(prefixAcl), getAcls(allTopicAcls))
@ -321,7 +322,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete all prefixed ACLs: // Delete all prefixed ACLs:
deleted = client.deleteAcls(List(allPrefixedTopicAcls).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(allPrefixedTopicAcls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(prefixAcl.entry(), b.dataPlaneRequestProcessor.authorizer.get, prefixAcl.pattern()) waitAndVerifyRemovedAcl(prefixAcl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, prefixAcl.pattern())
} }
assertEquals(Set(anyAcl, acl2, fooAcl), getAcls(allTopicAcls)) assertEquals(Set(anyAcl, acl2, fooAcl), getAcls(allTopicAcls))
@ -331,7 +332,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
deleted = client.deleteAcls(List(allTopicAcls).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(allTopicAcls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
Set(anyAcl, acl2, fooAcl, prefixAcl).foreach(acl => Set(anyAcl, acl2, fooAcl, prefixAcl).foreach(acl =>
waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizer.get, acl.pattern()) waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, acl.pattern())
) )
} }
assertEquals(Set(), getAcls(allTopicAcls)) assertEquals(Set(), getAcls(allTopicAcls))
@ -357,7 +358,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete only (legacy) ACLs on 'mytopic2' topic // Delete only (legacy) ACLs on 'mytopic2' topic
var deleted = client.deleteAcls(List(legacyMyTopic2Acls).asJava).all().get().asScala.toSet var deleted = client.deleteAcls(List(legacyMyTopic2Acls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(acl2.entry(), b.dataPlaneRequestProcessor.authorizer.get, acl2.pattern()) waitAndVerifyRemovedAcl(acl2.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, acl2.pattern())
} }
assertEquals(Set(anyAcl, fooAcl, prefixAcl), getAcls(allTopicAcls)) assertEquals(Set(anyAcl, fooAcl, prefixAcl), getAcls(allTopicAcls))
@ -366,7 +367,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
// Delete only (legacy) ACLs on '*' topic // Delete only (legacy) ACLs on '*' topic
deleted = client.deleteAcls(List(legacyAnyTopicAcls).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(legacyAnyTopicAcls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(anyAcl.entry(), b.dataPlaneRequestProcessor.authorizer.get, anyAcl.pattern()) waitAndVerifyRemovedAcl(anyAcl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, anyAcl.pattern())
} }
assertEquals(Set(acl2, fooAcl, prefixAcl), getAcls(allTopicAcls)) assertEquals(Set(acl2, fooAcl, prefixAcl), getAcls(allTopicAcls))
@ -376,7 +377,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
deleted = client.deleteAcls(List(legacyAllTopicAcls).asJava).all().get().asScala.toSet deleted = client.deleteAcls(List(legacyAllTopicAcls).asJava).all().get().asScala.toSet
brokers.foreach { b => brokers.foreach { b =>
Set(anyAcl, acl2, fooAcl).foreach(acl => Set(anyAcl, acl2, fooAcl).foreach(acl =>
waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizer.get, acl.pattern()) waitAndVerifyRemovedAcl(acl.entry(), b.dataPlaneRequestProcessor.authorizerPlugin.get, acl.pattern())
) )
} }
assertEquals(Set(), getAcls(legacyAllTopicAcls)) assertEquals(Set(), getAcls(legacyAllTopicAcls))
@ -542,7 +543,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
val ace = clusterAcl(permissionType, operation) val ace = clusterAcl(permissionType, operation)
superUserAdmin.createAcls(List(new AclBinding(clusterResourcePattern, ace)).asJava) superUserAdmin.createAcls(List(new AclBinding(clusterResourcePattern, ace)).asJava)
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) waitAndVerifyAcl(ace, b.dataPlaneRequestProcessor.authorizerPlugin.get, clusterResourcePattern)
} }
} }
@ -551,7 +552,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
superUserAdmin.deleteAcls(List(new AclBinding(clusterResourcePattern, ace).toFilter).asJava).values superUserAdmin.deleteAcls(List(new AclBinding(clusterResourcePattern, ace).toFilter).asJava).values
brokers.foreach { b => brokers.foreach { b =>
waitAndVerifyRemovedAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) waitAndVerifyRemovedAcl(ace, b.dataPlaneRequestProcessor.authorizerPlugin.get, clusterResourcePattern)
} }
} }
@ -709,11 +710,11 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
} }
def waitAndVerifyRemovedAcl(expectedToRemoved: AccessControlEntry, def waitAndVerifyRemovedAcl(expectedToRemoved: AccessControlEntry,
authorizer: JAuthorizer, authorizerPlugin: Plugin[JAuthorizer],
resource: ResourcePattern, resource: ResourcePattern,
accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = {
val newLine = scala.util.Properties.lineSeparator val newLine = scala.util.Properties.lineSeparator
val authorizer = authorizerPlugin.get
val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter)
waitUntilTrue(() => !authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expectedToRemoved), waitUntilTrue(() => !authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expectedToRemoved),
s"expected acl to be removed : $expectedToRemoved" + s"expected acl to be removed : $expectedToRemoved" +
@ -722,11 +723,11 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
} }
def waitAndVerifyAcl(expected: AccessControlEntry, def waitAndVerifyAcl(expected: AccessControlEntry,
authorizer: JAuthorizer, authorizerPlugin: Plugin[JAuthorizer],
resource: ResourcePattern, resource: ResourcePattern,
accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = {
val newLine = scala.util.Properties.lineSeparator val newLine = scala.util.Properties.lineSeparator
val authorizer = authorizerPlugin.get
val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter)
waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expected), waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expected),
s"expected to contain acl: $expected" + s"expected to contain acl: $expected" +

View File

@ -1229,9 +1229,9 @@ class KRaftClusterTest {
def assertFoobarValue(expected: Int): Unit = { def assertFoobarValue(expected: Int): Unit = {
TestUtils.retry(60000) { TestUtils.retry(60000) {
assertEquals(expected, cluster.controllers().values().iterator().next(). assertEquals(expected, cluster.controllers().values().iterator().next().
authorizer.get.asInstanceOf[FakeConfigurableAuthorizer].foobar.get()) authorizerPlugin.get.get.asInstanceOf[FakeConfigurableAuthorizer].foobar.get())
assertEquals(expected, cluster.brokers().values().iterator().next(). assertEquals(expected, cluster.brokers().values().iterator().next().
authorizer.get.asInstanceOf[FakeConfigurableAuthorizer].foobar.get()) authorizerPlugin.get.get.asInstanceOf[FakeConfigurableAuthorizer].foobar.get())
} }
} }

View File

@ -223,9 +223,9 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
result.exception.ifPresent { e => throw e } result.exception.ifPresent { e => throw e }
} }
val aclFilter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY) val aclFilter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY)
(brokers.map(_.authorizer.get) ++ controllerServers.map(_.authorizer.get)).foreach { (brokers.map(_.authorizerPlugin.get) ++ controllerServers.map(_.authorizerPlugin.get)).foreach {
authorizer => waitAndVerifyAcls( authorizer => waitAndVerifyAcls(
authorizer.acls(aclFilter).asScala.map(_.entry).toSet ++ acls, authorizer.get.acls(aclFilter).asScala.map(_.entry).toSet ++ acls,
authorizer, resource) authorizer, resource)
} }
} }
@ -239,9 +239,9 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
result.exception.ifPresent { e => throw e } result.exception.ifPresent { e => throw e }
} }
val aclFilter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY) val aclFilter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY)
(brokers.map(_.authorizer.get) ++ controllerServers.map(_.authorizer.get)).foreach { (brokers.map(_.authorizerPlugin.get) ++ controllerServers.map(_.authorizerPlugin.get)).foreach {
authorizer => waitAndVerifyAcls( authorizer => waitAndVerifyAcls(
authorizer.acls(aclFilter).asScala.map(_.entry).toSet -- acls, authorizer.get.acls(aclFilter).asScala.map(_.entry).toSet -- acls,
authorizer, resource) authorizer, resource)
} }
} }

View File

@ -23,6 +23,8 @@ import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl._
import org.apache.kafka.common.errors.ApiException import org.apache.kafka.common.errors.ApiException
import org.apache.kafka.common.metrics.{Metrics, PluginMetrics}
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl
import org.apache.kafka.common.requests.RequestContext import org.apache.kafka.common.requests.RequestContext
import org.apache.kafka.common.resource.PatternType.{LITERAL, MATCH, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, MATCH, PREFIXED}
import org.apache.kafka.common.resource.Resource.CLUSTER_NAME import org.apache.kafka.common.resource.Resource.CLUSTER_NAME
@ -82,9 +84,9 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
val props = properties val props = properties
config = KafkaConfig.fromProps(props) config = KafkaConfig.fromProps(props)
authorizer1 = createAuthorizer() authorizer1 = createAuthorizer()
configureAuthorizer(authorizer1, config.originals) configureAuthorizer(authorizer1, config.originals, new PluginMetricsImpl(new Metrics(), util.Map.of()))
authorizer2 = createAuthorizer() authorizer2 = createAuthorizer()
configureAuthorizer(authorizer2, config.originals) configureAuthorizer(authorizer2, config.originals, new PluginMetricsImpl(new Metrics(), util.Map.of()))
resource = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL) resource = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL)
} }
@ -297,7 +299,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
val testAuthorizer = createAuthorizer() val testAuthorizer = createAuthorizer()
try { try {
configureAuthorizer(testAuthorizer, cfg.originals) configureAuthorizer(testAuthorizer, cfg.originals, new PluginMetricsImpl(new Metrics(), util.Map.of()))
assertTrue(authorize(testAuthorizer, requestContext, READ, resource), assertTrue(authorize(testAuthorizer, requestContext, READ, resource),
"when acls = null or [], authorizer should allow op with allow.everyone = true.") "when acls = null or [], authorizer should allow op with allow.everyone = true.")
} finally { } finally {
@ -315,7 +317,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
val testAuthorizer = createAuthorizer() val testAuthorizer = createAuthorizer()
try { try {
configureAuthorizer(testAuthorizer, cfg.originals) configureAuthorizer(testAuthorizer, cfg.originals, new PluginMetricsImpl(new Metrics(), util.Map.of()))
assertTrue(authorize(testAuthorizer, requestContext, READ, resource), assertTrue(authorize(testAuthorizer, requestContext, READ, resource),
"when acls = null or [], authorizer should allow op with allow.everyone = true.") "when acls = null or [], authorizer should allow op with allow.everyone = true.")
} finally { } finally {
@ -366,7 +368,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
//test remove all acls for resource //test remove all acls for resource
removeAcls(authorizer1, Set.empty, resource) removeAcls(authorizer1, Set.empty, resource)
TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], authorizer1, resource) TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], authorizer1, resource, AccessControlEntryFilter.ANY)
acls = changeAclAndVerify(Set.empty, Set(acl1), Set.empty) acls = changeAclAndVerify(Set.empty, Set(acl1), Set.empty)
changeAclAndVerify(acls, Set.empty, acls) changeAclAndVerify(acls, Set.empty, acls)
@ -385,7 +387,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
addAcls(authorizer1, Set(acl1), commonResource) addAcls(authorizer1, Set(acl1), commonResource)
addAcls(authorizer1, Set(acl2), commonResource) addAcls(authorizer1, Set(acl2), commonResource)
TestUtils.waitAndVerifyAcls(Set(acl1, acl2), authorizer1, commonResource) TestUtils.waitAndVerifyAcls(Set(acl1, acl2), authorizer1, commonResource, AccessControlEntryFilter.ANY)
} }
/** /**
@ -629,7 +631,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
val authorizer: Authorizer = createAuthorizer() val authorizer: Authorizer = createAuthorizer()
try { try {
configureAuthorizer(authorizer, cfg.originals) configureAuthorizer(authorizer, cfg.originals, new PluginMetricsImpl(new Metrics(), util.Map.of()))
assertTrue(authorizeByResourceType(authorizer, requestContext, READ, resource.resourceType()), assertTrue(authorizeByResourceType(authorizer, requestContext, READ, resource.resourceType()),
"If allow.everyone.if.no.acl.found = true, caller should have read access to at least one topic") "If allow.everyone.if.no.acl.found = true, caller should have read access to at least one topic")
assertTrue(authorizeByResourceType(authorizer, requestContext, WRITE, resource.resourceType()), assertTrue(authorizeByResourceType(authorizer, requestContext, WRITE, resource.resourceType()),
@ -655,7 +657,7 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
acls --= removedAcls acls --= removedAcls
} }
TestUtils.waitAndVerifyAcls(acls, authorizer1, resource) TestUtils.waitAndVerifyAcls(acls, authorizer1, resource, AccessControlEntryFilter.ANY)
acls acls
} }
@ -689,13 +691,16 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
} }
def configureAuthorizer(authorizer: Authorizer, def configureAuthorizer(authorizer: Authorizer,
configs: util.Map[String, AnyRef]): Unit = { configs: util.Map[String, AnyRef],
configureStandardAuthorizer(authorizer.asInstanceOf[StandardAuthorizer], configs) pluginMetrics: PluginMetrics): Unit = {
configureStandardAuthorizer(authorizer.asInstanceOf[StandardAuthorizer], configs, pluginMetrics)
} }
def configureStandardAuthorizer(standardAuthorizer: StandardAuthorizer, def configureStandardAuthorizer(standardAuthorizer: StandardAuthorizer,
configs: util.Map[String, AnyRef]): Unit = { configs: util.Map[String, AnyRef],
pluginMetrics: PluginMetrics): Unit = {
standardAuthorizer.configure(configs) standardAuthorizer.configure(configs)
standardAuthorizer.withPluginMetrics(pluginMetrics)
initializeStandardAuthorizer(standardAuthorizer, new AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT))) initializeStandardAuthorizer(standardAuthorizer, new AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT)))
} }

View File

@ -23,6 +23,7 @@ import org.apache.kafka.clients.admin.EndpointType
import java.net.InetAddress import java.net.InetAddress
import java.util import java.util
import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.message.{DescribeClusterRequestData, DescribeClusterResponseData} import org.apache.kafka.common.message.{DescribeClusterRequestData, DescribeClusterResponseData}
import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection
import org.apache.kafka.common.network.{ClientInformation, ListenerName} import org.apache.kafka.common.network.{ClientInformation, ListenerName}
@ -71,6 +72,7 @@ class AuthHelperTest {
@Test @Test
def testAuthorize(): Unit = { def testAuthorize(): Unit = {
val authorizer: Authorizer = mock(classOf[Authorizer]) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val operation = AclOperation.WRITE val operation = AclOperation.WRITE
val resourceType = ResourceType.TOPIC val resourceType = ResourceType.TOPIC
@ -88,7 +90,7 @@ class AuthHelperTest {
when(authorizer.authorize(requestContext, expectedActions.asJava)) when(authorizer.authorize(requestContext, expectedActions.asJava))
.thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava)
val result = new AuthHelper(Some(authorizer)).authorize( val result = new AuthHelper(Some(authorizerPlugin)).authorize(
requestContext, operation, resourceType, resourceName) requestContext, operation, resourceType, resourceName)
verify(authorizer).authorize(requestContext, expectedActions.asJava) verify(authorizer).authorize(requestContext, expectedActions.asJava)
@ -99,6 +101,7 @@ class AuthHelperTest {
@Test @Test
def testFilterByAuthorized(): Unit = { def testFilterByAuthorized(): Unit = {
val authorizer: Authorizer = mock(classOf[Authorizer]) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val operation = AclOperation.WRITE val operation = AclOperation.WRITE
val resourceType = ResourceType.TOPIC val resourceType = ResourceType.TOPIC
@ -132,7 +135,7 @@ class AuthHelperTest {
}.asJava }.asJava
} }
val result = new AuthHelper(Some(authorizer)).filterByAuthorized( val result = new AuthHelper(Some(authorizerPlugin)).filterByAuthorized(
requestContext, requestContext,
operation, operation,
resourceType, resourceType,
@ -149,7 +152,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseV1WithUnknownEndpointType(): Unit = { def testComputeDescribeClusterResponseV1WithUnknownEndpointType(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(123.toByte), 1) new DescribeClusterRequestData().setEndpointType(123.toByte), 1)
val responseData = authHelper.computeDescribeClusterResponse(request, val responseData = authHelper.computeDescribeClusterResponse(request,
@ -164,7 +169,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseV0WithUnknownEndpointType(): Unit = { def testComputeDescribeClusterResponseV0WithUnknownEndpointType(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(123.toByte), 0) new DescribeClusterRequestData().setEndpointType(123.toByte), 0)
val responseData = authHelper.computeDescribeClusterResponse(request, val responseData = authHelper.computeDescribeClusterResponse(request,
@ -179,7 +186,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseV1WithUnexpectedEndpointType(): Unit = { def testComputeDescribeClusterResponseV1WithUnexpectedEndpointType(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(EndpointType.BROKER.id()), 1) new DescribeClusterRequestData().setEndpointType(EndpointType.BROKER.id()), 1)
val responseData = authHelper.computeDescribeClusterResponse(request, val responseData = authHelper.computeDescribeClusterResponse(request,
@ -194,7 +203,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseV0WithUnexpectedEndpointType(): Unit = { def testComputeDescribeClusterResponseV0WithUnexpectedEndpointType(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(EndpointType.BROKER.id()), 0) new DescribeClusterRequestData().setEndpointType(EndpointType.BROKER.id()), 0)
val responseData = authHelper.computeDescribeClusterResponse(request, val responseData = authHelper.computeDescribeClusterResponse(request,
@ -209,7 +220,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseWhereControllerIsNotFound(): Unit = { def testComputeDescribeClusterResponseWhereControllerIsNotFound(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(EndpointType.CONTROLLER.id()), 1) new DescribeClusterRequestData().setEndpointType(EndpointType.CONTROLLER.id()), 1)
val responseData = authHelper.computeDescribeClusterResponse(request, val responseData = authHelper.computeDescribeClusterResponse(request,
@ -226,7 +239,9 @@ class AuthHelperTest {
@Test @Test
def testComputeDescribeClusterResponseSuccess(): Unit = { def testComputeDescribeClusterResponseSuccess(): Unit = {
val authHelper = new AuthHelper(Some(mock(classOf[Authorizer]))) val authorizer: Authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val authHelper = new AuthHelper(Some(authorizerPlugin))
val request = newMockDescribeClusterRequest( val request = newMockDescribeClusterRequest(
new DescribeClusterRequestData().setEndpointType(EndpointType.CONTROLLER.id()), 1) new DescribeClusterRequestData().setEndpointType(EndpointType.CONTROLLER.id()), 1)
val nodes = new DescribeClusterBrokerCollection( val nodes = new DescribeClusterBrokerCollection(

View File

@ -26,7 +26,7 @@ import org.apache.kafka.common.Uuid.ZERO_UUID
import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.{Plugin, Topic}
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResource => OldAlterConfigsResource, AlterConfigsResourceCollection => OldAlterConfigsResourceCollection, AlterableConfig => OldAlterableConfig, AlterableConfigCollection => OldAlterableConfigCollection} import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResource => OldAlterConfigsResource, AlterConfigsResourceCollection => OldAlterConfigsResourceCollection, AlterableConfig => OldAlterableConfig, AlterableConfigCollection => OldAlterableConfigCollection}
import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse} import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse}
@ -151,7 +151,7 @@ class ControllerApisTest {
private var controllerApis: ControllerApis = _ private var controllerApis: ControllerApis = _
private def createControllerApis(authorizer: Option[Authorizer], private def createControllerApis(authorizer: Option[Plugin[Authorizer]],
controller: Controller, controller: Controller,
props: Properties = new Properties(), props: Properties = new Properties(),
throttle: Boolean = false): ControllerApis = { throttle: Boolean = false): ControllerApis = {
@ -200,7 +200,7 @@ class ControllerApisTest {
requestChannelMetrics) requestChannelMetrics)
} }
def createDenyAllAuthorizer(): Authorizer = { def createDenyAllAuthorizer(): Plugin[Authorizer] = {
val authorizer = mock(classOf[Authorizer]) val authorizer = mock(classOf[Authorizer])
when(authorizer.authorize( when(authorizer.authorize(
any(classOf[AuthorizableRequestContext]), any(classOf[AuthorizableRequestContext]),
@ -208,7 +208,7 @@ class ControllerApisTest {
)).thenReturn( )).thenReturn(
singletonList(AuthorizationResult.DENIED) singletonList(AuthorizationResult.DENIED)
) )
authorizer Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
} }
@Test @Test
@ -1009,7 +1009,8 @@ class ControllerApisTest {
.newInitialTopic("foo", Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q")) .newInitialTopic("foo", Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q"))
.build() .build()
val authorizer = mock(classOf[Authorizer]) val authorizer = mock(classOf[Authorizer])
controllerApis = createControllerApis(Some(authorizer), controller) val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
controllerApis = createControllerApis(Some(authorizerPlugin), controller)
val requestData = new CreatePartitionsRequestData() val requestData = new CreatePartitionsRequestData()
requestData.topics().add(new CreatePartitionsTopic().setName("foo").setAssignments(null).setCount(2)) requestData.topics().add(new CreatePartitionsTopic().setName("foo").setAssignments(null).setCount(2))
requestData.topics().add(new CreatePartitionsTopic().setName("bar").setAssignments(null).setCount(10)) requestData.topics().add(new CreatePartitionsTopic().setName("bar").setAssignments(null).setCount(10))
@ -1069,8 +1070,9 @@ class ControllerApisTest {
@Test @Test
def testElectLeadersAuthorization(): Unit = { def testElectLeadersAuthorization(): Unit = {
val authorizer = mock(classOf[Authorizer]) val authorizer = mock(classOf[Authorizer])
val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
val controller = mock(classOf[Controller]) val controller = mock(classOf[Controller])
controllerApis = createControllerApis(Some(authorizer), controller) controllerApis = createControllerApis(Some(authorizerPlugin), controller)
val request = new ElectLeadersRequest.Builder( val request = new ElectLeadersRequest.Builder(
ElectionType.PREFERRED, ElectionType.PREFERRED,
null, null,
@ -1213,7 +1215,8 @@ class ControllerApisTest {
def testAssignReplicasToDirs(): Unit = { def testAssignReplicasToDirs(): Unit = {
val controller = mock(classOf[Controller]) val controller = mock(classOf[Controller])
val authorizer = mock(classOf[Authorizer]) val authorizer = mock(classOf[Authorizer])
controllerApis = createControllerApis(Some(authorizer), controller) val authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
controllerApis = createControllerApis(Some(authorizerPlugin), controller)
val request = new AssignReplicasToDirsRequest.Builder(new AssignReplicasToDirsRequestData()).build() val request = new AssignReplicasToDirsRequest.Builder(new AssignReplicasToDirsRequestData()).build()
when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(Collections.singletonList(new Action( when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(Collections.singletonList(new Action(

View File

@ -28,6 +28,7 @@ import kafka.utils.TestUtils
import org.apache.kafka.common.{Endpoint, Reconfigurable} import org.apache.kafka.common.{Endpoint, Reconfigurable}
import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter}
import org.apache.kafka.common.config.{ConfigException, SslConfigs} import org.apache.kafka.common.config.{ConfigException, SslConfigs}
import org.apache.kafka.common.internals.Plugin
import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.metrics.{JmxReporter, Metrics}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
@ -480,7 +481,8 @@ class DynamicBrokerConfigTest {
when(kafkaServer.logManager).thenReturn(logManager) when(kafkaServer.logManager).thenReturn(logManager)
val authorizer = new TestAuthorizer val authorizer = new TestAuthorizer
when(kafkaServer.authorizer).thenReturn(Some(authorizer)) val authorizerPlugin: Plugin[Authorizer] = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
when(kafkaServer.authorizerPlugin).thenReturn(Some(authorizerPlugin))
kafkaServer.config.dynamicConfig.addReconfigurables(kafkaServer) kafkaServer.config.dynamicConfig.addReconfigurables(kafkaServer)
props.put("super.users", "User:admin") props.put("super.users", "User:admin")
@ -522,7 +524,8 @@ class DynamicBrokerConfigTest {
when(controllerServer.socketServer).thenReturn(socketServer) when(controllerServer.socketServer).thenReturn(socketServer)
val authorizer = new TestAuthorizer val authorizer = new TestAuthorizer
when(controllerServer.authorizer).thenReturn(Some(authorizer)) val authorizerPlugin: Plugin[Authorizer] = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
when(controllerServer.authorizerPlugin).thenReturn(Some(authorizerPlugin))
controllerServer.config.dynamicConfig.addReconfigurables(controllerServer) controllerServer.config.dynamicConfig.addReconfigurables(controllerServer)
props.put("super.users", "User:admin") props.put("super.users", "User:admin")
@ -567,7 +570,8 @@ class DynamicBrokerConfigTest {
when(controllerServer.socketServer).thenReturn(socketServer) when(controllerServer.socketServer).thenReturn(socketServer)
val authorizer = new TestAuthorizer val authorizer = new TestAuthorizer
when(controllerServer.authorizer).thenReturn(Some(authorizer)) val authorizerPlugin: Plugin[Authorizer] = Plugin.wrapInstance(authorizer, null, "authorizer.class.name")
when(controllerServer.authorizerPlugin).thenReturn(Some(authorizerPlugin))
controllerServer.config.dynamicConfig.addReconfigurables(controllerServer) controllerServer.config.dynamicConfig.addReconfigurables(controllerServer)
props.put("super.users", "User:admin") props.put("super.users", "User:admin")

View File

@ -32,7 +32,7 @@ import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER} import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER}
import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException} import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException}
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.{Plugin, Topic}
import org.apache.kafka.common.internals.Topic.SHARE_GROUP_STATE_TOPIC_NAME import org.apache.kafka.common.internals.Topic.SHARE_GROUP_STATE_TOPIC_NAME
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.{AddPartitionsToTxnTopic, AddPartitionsToTxnTopicCollection, AddPartitionsToTxnTransaction, AddPartitionsToTxnTransactionCollection} import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.{AddPartitionsToTxnTopic, AddPartitionsToTxnTopicCollection, AddPartitionsToTxnTransaction, AddPartitionsToTxnTransactionCollection}
@ -199,7 +199,7 @@ class KafkaApisTest extends Logging {
configRepository = configRepository, configRepository = configRepository,
metadataCache = metadataCache, metadataCache = metadataCache,
metrics = metrics, metrics = metrics,
authorizer = authorizer, authorizerPlugin = authorizer.map(Plugin.wrapInstance(_, null, "authorizer.class.name")),
quotas = quotas, quotas = quotas,
fetchManager = fetchManager, fetchManager = fetchManager,
sharePartitionManager = sharePartitionManager, sharePartitionManager = sharePartitionManager,

View File

@ -33,7 +33,7 @@ import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.{ConfigException, ConfigResource}
import org.apache.kafka.common.errors.{OperationNotAttemptedException, TopicExistsException, UnknownTopicOrPartitionException} import org.apache.kafka.common.errors.{OperationNotAttemptedException, TopicExistsException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.header.Header import org.apache.kafka.common.header.Header
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.{Plugin, Topic}
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ClientInformation, ConnectionMode, ListenerName} import org.apache.kafka.common.network.{ClientInformation, ConnectionMode, ListenerName}
@ -1122,11 +1122,17 @@ object TestUtils extends Logging {
} }
def waitAndVerifyAcls(expected: Set[AccessControlEntry], def waitAndVerifyAcls(expected: Set[AccessControlEntry],
authorizer: JAuthorizer, authorizerPlugin: Plugin[JAuthorizer],
resource: ResourcePattern, resource: ResourcePattern,
accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = {
val newLine = scala.util.Properties.lineSeparator waitAndVerifyAcls(expected, authorizerPlugin.get, resource, accessControlEntryFilter)
}
def waitAndVerifyAcls(expected: Set[AccessControlEntry],
authorizer: JAuthorizer,
resource: ResourcePattern,
accessControlEntryFilter: AccessControlEntryFilter): Unit = {
val newLine = scala.util.Properties.lineSeparator
val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter)
waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet == expected, waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet == expected,
s"expected acls:${expected.mkString(newLine + "\t", newLine + "\t", newLine)}" + s"expected acls:${expected.mkString(newLine + "\t", newLine + "\t", newLine)}" +
@ -1431,12 +1437,12 @@ object TestUtils extends Logging {
controllers: Seq[ControllerServer], controllers: Seq[ControllerServer],
): JAuthorizer = { ): JAuthorizer = {
if (controllers.isEmpty) { if (controllers.isEmpty) {
brokers.head.authorizer.get brokers.head.authorizerPlugin.get.get
} else { } else {
var result: JAuthorizer = null var result: JAuthorizer = null
TestUtils.retry(120000) { TestUtils.retry(120000) {
val active = controllers.filter(_.controller.isActive).head val active = controllers.filter(_.controller.isActive).head
result = active.authorizer.get result = active.authorizerPlugin.get.get
} }
result result
} }

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.errors.NotCoordinatorException; import org.apache.kafka.common.errors.NotCoordinatorException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
@ -144,7 +145,7 @@ public class GroupCoordinatorService implements GroupCoordinator {
private GroupCoordinatorMetrics groupCoordinatorMetrics; private GroupCoordinatorMetrics groupCoordinatorMetrics;
private GroupConfigManager groupConfigManager; private GroupConfigManager groupConfigManager;
private Persister persister; private Persister persister;
private Optional<Authorizer> authorizer; private Optional<Plugin<Authorizer>> authorizerPlugin;
public Builder( public Builder(
int nodeId, int nodeId,
@ -194,8 +195,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
return this; return this;
} }
public Builder withAuthorizer(Optional<Authorizer> authorizer) { public Builder withAuthorizerPlugin(Optional<Plugin<Authorizer>> authorizerPlugin) {
this.authorizer = authorizer; this.authorizerPlugin = authorizerPlugin;
return this; return this;
} }
@ -209,14 +210,14 @@ public class GroupCoordinatorService implements GroupCoordinator {
requireNonNull(groupCoordinatorMetrics, new IllegalArgumentException("GroupCoordinatorMetrics must be set.")); requireNonNull(groupCoordinatorMetrics, new IllegalArgumentException("GroupCoordinatorMetrics must be set."));
requireNonNull(groupConfigManager, new IllegalArgumentException("GroupConfigManager must be set.")); requireNonNull(groupConfigManager, new IllegalArgumentException("GroupConfigManager must be set."));
requireNonNull(persister, new IllegalArgumentException("Persister must be set.")); requireNonNull(persister, new IllegalArgumentException("Persister must be set."));
requireNonNull(authorizer, new IllegalArgumentException("Authorizer must be set.")); requireNonNull(authorizerPlugin, new IllegalArgumentException("Authorizer must be set."));
String logPrefix = String.format("GroupCoordinator id=%d", nodeId); String logPrefix = String.format("GroupCoordinator id=%d", nodeId);
LogContext logContext = new LogContext(String.format("[%s] ", logPrefix)); LogContext logContext = new LogContext(String.format("[%s] ", logPrefix));
CoordinatorShardBuilderSupplier<GroupCoordinatorShard, CoordinatorRecord> supplier = () -> CoordinatorShardBuilderSupplier<GroupCoordinatorShard, CoordinatorRecord> supplier = () ->
new GroupCoordinatorShard.Builder(config, groupConfigManager) new GroupCoordinatorShard.Builder(config, groupConfigManager)
.withAuthorizer(authorizer); .withAuthorizerPlugin(authorizerPlugin);
CoordinatorEventProcessor processor = new MultiThreadedEventProcessor( CoordinatorEventProcessor processor = new MultiThreadedEventProcessor(
logContext, logContext,

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.errors.GroupNotEmptyException;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
@ -163,7 +164,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
private CoordinatorExecutor<CoordinatorRecord> executor; private CoordinatorExecutor<CoordinatorRecord> executor;
private CoordinatorMetrics coordinatorMetrics; private CoordinatorMetrics coordinatorMetrics;
private TopicPartition topicPartition; private TopicPartition topicPartition;
private Optional<Authorizer> authorizer; private Optional<Plugin<Authorizer>> authorizerPlugin;
public Builder( public Builder(
GroupCoordinatorConfig config, GroupCoordinatorConfig config,
@ -227,10 +228,10 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
return this; return this;
} }
public CoordinatorShardBuilder<GroupCoordinatorShard, CoordinatorRecord> withAuthorizer( public CoordinatorShardBuilder<GroupCoordinatorShard, CoordinatorRecord> withAuthorizerPlugin(
Optional<Authorizer> authorizer Optional<Plugin<Authorizer>> authorizerPlugin
) { ) {
this.authorizer = authorizer; this.authorizerPlugin = authorizerPlugin;
return this; return this;
} }
@ -254,7 +255,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
throw new IllegalArgumentException("TopicPartition must be set."); throw new IllegalArgumentException("TopicPartition must be set.");
if (groupConfigManager == null) if (groupConfigManager == null)
throw new IllegalArgumentException("GroupConfigManager must be set."); throw new IllegalArgumentException("GroupConfigManager must be set.");
if (authorizer == null) if (authorizerPlugin == null)
throw new IllegalArgumentException("Authorizer must be set."); throw new IllegalArgumentException("Authorizer must be set.");
GroupCoordinatorMetricsShard metricsShard = ((GroupCoordinatorMetrics) coordinatorMetrics) GroupCoordinatorMetricsShard metricsShard = ((GroupCoordinatorMetrics) coordinatorMetrics)
@ -269,7 +270,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
.withConfig(config) .withConfig(config)
.withGroupConfigManager(groupConfigManager) .withGroupConfigManager(groupConfigManager)
.withGroupCoordinatorMetricsShard(metricsShard) .withGroupCoordinatorMetricsShard(metricsShard)
.withAuthorizer(authorizer) .withAuthorizerPlugin(authorizerPlugin)
.build(); .build();
OffsetMetadataManager offsetMetadataManager = new OffsetMetadataManager.Builder() OffsetMetadataManager offsetMetadataManager = new OffsetMetadataManager.Builder()

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnreleasedInstanceIdException; import org.apache.kafka.common.errors.UnreleasedInstanceIdException;
import org.apache.kafka.common.errors.UnsupportedAssignorException; import org.apache.kafka.common.errors.UnsupportedAssignorException;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
@ -301,7 +302,7 @@ public class GroupMetadataManager {
private MetadataImage metadataImage = null; private MetadataImage metadataImage = null;
private ShareGroupPartitionAssignor shareGroupAssignor = null; private ShareGroupPartitionAssignor shareGroupAssignor = null;
private GroupCoordinatorMetricsShard metrics; private GroupCoordinatorMetricsShard metrics;
private Optional<Authorizer> authorizer = null; private Optional<Plugin<Authorizer>> authorizerPlugin = null;
private List<TaskAssignor> streamsGroupAssignors = null; private List<TaskAssignor> streamsGroupAssignors = null;
Builder withLogContext(LogContext logContext) { Builder withLogContext(LogContext logContext) {
@ -359,8 +360,8 @@ public class GroupMetadataManager {
return this; return this;
} }
Builder withAuthorizer(Optional<Authorizer> authorizer) { Builder withAuthorizerPlugin(Optional<Plugin<Authorizer>> authorizerPlugin) {
this.authorizer = authorizer; this.authorizerPlugin = authorizerPlugin;
return this; return this;
} }
@ -369,7 +370,7 @@ public class GroupMetadataManager {
if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
if (metadataImage == null) metadataImage = MetadataImage.EMPTY; if (metadataImage == null) metadataImage = MetadataImage.EMPTY;
if (time == null) time = Time.SYSTEM; if (time == null) time = Time.SYSTEM;
if (authorizer == null) authorizer = Optional.empty(); if (authorizerPlugin == null) authorizerPlugin = Optional.empty();
if (timer == null) if (timer == null)
throw new IllegalArgumentException("Timer must be set."); throw new IllegalArgumentException("Timer must be set.");
@ -397,7 +398,7 @@ public class GroupMetadataManager {
config, config,
groupConfigManager, groupConfigManager,
shareGroupAssignor, shareGroupAssignor,
authorizer, authorizerPlugin,
streamsGroupAssignors streamsGroupAssignors
); );
} }
@ -526,7 +527,7 @@ public class GroupMetadataManager {
/** /**
* The authorizer to validate the regex subscription topics. * The authorizer to validate the regex subscription topics.
*/ */
private final Optional<Authorizer> authorizer; private final Optional<Plugin<Authorizer>> authorizerPlugin;
private GroupMetadataManager( private GroupMetadataManager(
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
@ -539,7 +540,7 @@ public class GroupMetadataManager {
GroupCoordinatorConfig config, GroupCoordinatorConfig config,
GroupConfigManager groupConfigManager, GroupConfigManager groupConfigManager,
ShareGroupPartitionAssignor shareGroupAssignor, ShareGroupPartitionAssignor shareGroupAssignor,
Optional<Authorizer> authorizer, Optional<Plugin<Authorizer>> authorizerPlugin,
List<TaskAssignor> streamsGroupAssignors List<TaskAssignor> streamsGroupAssignors
) { ) {
this.logContext = logContext; this.logContext = logContext;
@ -561,7 +562,7 @@ public class GroupMetadataManager {
this.shareGroupPartitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0); this.shareGroupPartitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
this.groupConfigManager = groupConfigManager; this.groupConfigManager = groupConfigManager;
this.shareGroupAssignor = shareGroupAssignor; this.shareGroupAssignor = shareGroupAssignor;
this.authorizer = authorizer; this.authorizerPlugin = authorizerPlugin;
this.streamsGroupAssignors = streamsGroupAssignors.stream().collect(Collectors.toMap(TaskAssignor::name, Function.identity())); this.streamsGroupAssignors = streamsGroupAssignors.stream().collect(Collectors.toMap(TaskAssignor::name, Function.identity()));
} }
@ -3313,7 +3314,7 @@ public class GroupMetadataManager {
Set<String> regexes = Collections.unmodifiableSet(subscribedRegularExpressions.keySet()); Set<String> regexes = Collections.unmodifiableSet(subscribedRegularExpressions.keySet());
executor.schedule( executor.schedule(
key, key,
() -> refreshRegularExpressions(context, groupId, log, time, metadataImage, authorizer, regexes), () -> refreshRegularExpressions(context, groupId, log, time, metadataImage, authorizerPlugin, regexes),
(result, exception) -> handleRegularExpressionsResult(groupId, memberId, result, exception) (result, exception) -> handleRegularExpressionsResult(groupId, memberId, result, exception)
); );
} }
@ -3331,7 +3332,7 @@ public class GroupMetadataManager {
* @param log The log instance. * @param log The log instance.
* @param time The time instance. * @param time The time instance.
* @param image The metadata image to use for listing the topics. * @param image The metadata image to use for listing the topics.
* @param authorizer The authorizer. * @param authorizerPlugin The authorizer.
* @param regexes The list of regular expressions that must be resolved. * @param regexes The list of regular expressions that must be resolved.
* @return The list of resolved regular expressions. * @return The list of resolved regular expressions.
* *
@ -3343,7 +3344,7 @@ public class GroupMetadataManager {
Logger log, Logger log,
Time time, Time time,
MetadataImage image, MetadataImage image,
Optional<Authorizer> authorizer, Optional<Plugin<Authorizer>> authorizerPlugin,
Set<String> regexes Set<String> regexes
) { ) {
long startTimeMs = time.milliseconds(); long startTimeMs = time.milliseconds();
@ -3374,7 +3375,7 @@ public class GroupMetadataManager {
filterTopicDescribeAuthorizedTopics( filterTopicDescribeAuthorizedTopics(
context, context,
authorizer, authorizerPlugin,
resolvedRegexes resolvedRegexes
); );
@ -3399,15 +3400,15 @@ public class GroupMetadataManager {
* that the member is authorized to describe. * that the member is authorized to describe.
* *
* @param context The request context. * @param context The request context.
* @param authorizer The authorizer. * @param authorizerPlugin The authorizer.
* @param resolvedRegexes The map of the regex pattern and its set of matched topics. * @param resolvedRegexes The map of the regex pattern and its set of matched topics.
*/ */
private static void filterTopicDescribeAuthorizedTopics( private static void filterTopicDescribeAuthorizedTopics(
RequestContext context, RequestContext context,
Optional<Authorizer> authorizer, Optional<Plugin<Authorizer>> authorizerPlugin,
Map<String, Set<String>> resolvedRegexes Map<String, Set<String>> resolvedRegexes
) { ) {
if (authorizer.isEmpty()) return; if (authorizerPlugin.isEmpty()) return;
Map<String, Integer> topicNameCount = new HashMap<>(); Map<String, Integer> topicNameCount = new HashMap<>();
resolvedRegexes.values().forEach(topicNames -> resolvedRegexes.values().forEach(topicNames ->
@ -3421,7 +3422,7 @@ public class GroupMetadataManager {
return new Action(DESCRIBE, resource, entry.getValue(), true, false); return new Action(DESCRIBE, resource, entry.getValue(), true, false);
}).collect(Collectors.toList()); }).collect(Collectors.toList());
List<AuthorizationResult> authorizationResults = authorizer.get().authorize(context, actions); List<AuthorizationResult> authorizationResults = authorizerPlugin.get().get().authorize(context, actions);
Set<String> deniedTopics = new HashSet<>(); Set<String> deniedTopics = new HashSet<>();
IntStream.range(0, actions.size()).forEach(i -> { IntStream.range(0, actions.size()).forEach(i -> {
if (authorizationResults.get(i) == AuthorizationResult.DENIED) { if (authorizationResults.get(i) == AuthorizationResult.DENIED) {

View File

@ -37,6 +37,7 @@ import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.errors.UnreleasedInstanceIdException; import org.apache.kafka.common.errors.UnreleasedInstanceIdException;
import org.apache.kafka.common.errors.UnsupportedAssignorException; import org.apache.kafka.common.errors.UnsupportedAssignorException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
@ -20267,13 +20268,14 @@ public class GroupMetadataManagerTest {
MockPartitionAssignor assignor = new MockPartitionAssignor("range"); MockPartitionAssignor assignor = new MockPartitionAssignor("range");
Authorizer authorizer = mock(Authorizer.class); Authorizer authorizer = mock(Authorizer.class);
Plugin<Authorizer> authorizerPlugin = Plugin.wrapInstance(authorizer, null, "authorizer.class.name");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(new MetadataImageBuilder() .withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6) .addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3) .addTopic(barTopicId, barTopicName, 3)
.build(12345L)) .build(12345L))
.withAuthorizer(authorizer) .withAuthorizerPlugin(authorizerPlugin)
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1) .withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE) .setState(MemberState.STABLE)

View File

@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
@ -472,7 +473,7 @@ public class GroupMetadataManagerTestContext {
private ShareGroupPartitionAssignor shareGroupAssignor = new MockPartitionAssignor("share"); private ShareGroupPartitionAssignor shareGroupAssignor = new MockPartitionAssignor("share");
private final List<ShareGroupBuilder> shareGroupBuilders = new ArrayList<>(); private final List<ShareGroupBuilder> shareGroupBuilders = new ArrayList<>();
private final Map<String, Object> config = new HashMap<>(); private final Map<String, Object> config = new HashMap<>();
private Optional<Authorizer> authorizer = Optional.empty(); private Optional<Plugin<Authorizer>> authorizerPlugin = Optional.empty();
private List<TaskAssignor> streamsGroupAssignors = Collections.singletonList(new MockTaskAssignor("mock")); private List<TaskAssignor> streamsGroupAssignors = Collections.singletonList(new MockTaskAssignor("mock"));
public Builder withConfig(String key, Object value) { public Builder withConfig(String key, Object value) {
@ -505,8 +506,8 @@ public class GroupMetadataManagerTestContext {
return this; return this;
} }
public Builder withAuthorizer(Authorizer authorizer) { public Builder withAuthorizerPlugin(Plugin<Authorizer> authorizerPlugin) {
this.authorizer = Optional.of(authorizer); this.authorizerPlugin = Optional.of(authorizerPlugin);
return this; return this;
} }
@ -544,7 +545,7 @@ public class GroupMetadataManagerTestContext {
.withGroupCoordinatorMetricsShard(metrics) .withGroupCoordinatorMetricsShard(metrics)
.withShareGroupAssignor(shareGroupAssignor) .withShareGroupAssignor(shareGroupAssignor)
.withGroupConfigManager(groupConfigManager) .withGroupConfigManager(groupConfigManager)
.withAuthorizer(authorizer) .withAuthorizerPlugin(authorizerPlugin)
.withStreamsGroupAssignors(streamsGroupAssignors) .withStreamsGroupAssignors(streamsGroupAssignors)
.build(), .build(),
groupConfigManager groupConfigManager

View File

@ -195,7 +195,7 @@ public class KRaftMetadataRequestBenchmark {
setConfigRepository(new MockConfigRepository()). setConfigRepository(new MockConfigRepository()).
setMetadataCache(metadataCache). setMetadataCache(metadataCache).
setMetrics(metrics). setMetrics(metrics).
setAuthorizer(Optional.empty()). setAuthorizerPlugin(Optional.empty()).
setQuotas(quotaManagers). setQuotas(quotaManagers).
setFetchManager(fetchManager). setFetchManager(fetchManager).
setSharePartitionManager(sharePartitionManager). setSharePartitionManager(sharePartitionManager).

View File

@ -23,6 +23,12 @@ import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.errors.NotControllerException; import org.apache.kafka.common.errors.NotControllerException;
import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.metrics.stats.WindowedCount;
import org.apache.kafka.common.utils.SecurityUtils; import org.apache.kafka.common.utils.SecurityUtils;
import org.apache.kafka.server.authorizer.Action; import org.apache.kafka.server.authorizer.Action;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
@ -38,16 +44,16 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage; import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.server.authorizer.AuthorizationResult.ALLOWED; import static org.apache.kafka.server.authorizer.AuthorizationResult.ALLOWED;
import static org.apache.kafka.server.authorizer.AuthorizationResult.DENIED; import static org.apache.kafka.server.authorizer.AuthorizationResult.DENIED;
/** /**
* The standard authorizer which is used in KRaft-based clusters if no other authorizer is * Built-in authorizer implementation that stores ACLs in the metadata log.
* configured.
*/ */
public class StandardAuthorizer implements ClusterMetadataAuthorizer { public class StandardAuthorizer implements ClusterMetadataAuthorizer, Monitorable {
public static final String SUPER_USERS_CONFIG = "super.users"; public static final String SUPER_USERS_CONFIG = "super.users";
public static final String ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG = "allow.everyone.if.no.acl.found"; public static final String ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG = "allow.everyone.if.no.acl.found";
@ -64,6 +70,8 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer {
*/ */
private volatile StandardAuthorizerData data = StandardAuthorizerData.createEmpty(); private volatile StandardAuthorizerData data = StandardAuthorizerData.createEmpty();
private AuthorizerMetrics authorizerMetrics;
@Override @Override
public void setAclMutator(AclMutator aclMutator) { public void setAclMutator(AclMutator aclMutator) {
this.data = data.copyWithNewAclMutator(aclMutator); this.data = data.copyWithNewAclMutator(aclMutator);
@ -141,6 +149,7 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer {
for (Action action : actions) { for (Action action : actions) {
AuthorizationResult result = curData.authorize(requestContext, action); AuthorizationResult result = curData.authorize(requestContext, action);
results.add(result); results.add(result);
authorizerMetrics.recordAuthorizerMetrics(result);
} }
return results; return results;
} }
@ -207,4 +216,45 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer {
if (configValue == null) return DENIED; if (configValue == null) return DENIED;
return Boolean.parseBoolean(configValue.toString().trim()) ? ALLOWED : DENIED; return Boolean.parseBoolean(configValue.toString().trim()) ? ALLOWED : DENIED;
} }
@Override
public void withPluginMetrics(PluginMetrics metrics) {
this.authorizerMetrics = new AuthorizerMetrics(metrics);
}
private class AuthorizerMetrics {
private final Sensor authorizationAllowedSensor;
private final Sensor authorizationDeniedSensor;
private final Sensor authorizationRequestSensor;
private AuthorizerMetrics(PluginMetrics metrics) {
authorizationAllowedSensor = metrics.addSensor("authorizer-authorization-allowed");
authorizationAllowedSensor.add(
metrics.metricName("authorization-allowed-rate-per-minute", "The number of authorization allowed per minute", Map.of()),
new Rate(TimeUnit.MINUTES, new WindowedCount()));
authorizationDeniedSensor = metrics.addSensor("authorizer-authorization-denied");
authorizationDeniedSensor.add(
metrics.metricName("authorization-denied-rate-per-minute", "The number of authorization denied per minute", Map.of()),
new Rate(TimeUnit.MINUTES, new WindowedCount()));
authorizationRequestSensor = metrics.addSensor("authorizer-authorization-request");
authorizationRequestSensor.add(
metrics.metricName("authorization-request-rate-per-minute", "The number of authorization request per minute", Map.of()),
new Rate(TimeUnit.MINUTES, new WindowedCount()));
metrics.addMetric(
metrics.metricName("acls-total-count", "The number of acls defined", Map.of()),
(Gauge<Integer>) (config, now) -> aclCount());
}
private void recordAuthorizerMetrics(AuthorizationResult authorizationResult) {
if (authorizationResult == ALLOWED) {
authorizationAllowedSensor.record();
} else {
authorizationDeniedSensor.record();
}
authorizationRequestSensor.record();
}
}
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.metadata.publisher; package org.apache.kafka.metadata.publisher;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataImage;
@ -36,14 +37,14 @@ public class AclPublisher implements MetadataPublisher {
private final int nodeId; private final int nodeId;
private final FaultHandler faultHandler; private final FaultHandler faultHandler;
private final String nodeType; private final String nodeType;
private final Optional<ClusterMetadataAuthorizer> authorizer; private final Optional<Plugin<Authorizer>> authorizer;
private boolean completedInitialLoad = false; private boolean completedInitialLoad = false;
public AclPublisher(int nodeId, FaultHandler faultHandler, String nodeType, Optional<Authorizer> authorizer) { public AclPublisher(int nodeId, FaultHandler faultHandler, String nodeType, Optional<Plugin<Authorizer>> authorizer) {
this.nodeId = nodeId; this.nodeId = nodeId;
this.faultHandler = faultHandler; this.faultHandler = faultHandler;
this.nodeType = nodeType; this.nodeType = nodeType;
this.authorizer = authorizer.filter(ClusterMetadataAuthorizer.class::isInstance).map(ClusterMetadataAuthorizer.class::cast); this.authorizer = authorizer.filter(plugin -> plugin.get() instanceof ClusterMetadataAuthorizer);
this.log = new LogContext(name()).logger(AclPublisher.class); this.log = new LogContext(name()).logger(AclPublisher.class);
} }
@ -63,7 +64,8 @@ public class AclPublisher implements MetadataPublisher {
// we want to apply those changes in that order, not the reverse order! Otherwise // we want to apply those changes in that order, not the reverse order! Otherwise
// there could be a window during which incorrect authorization results are returned. // there could be a window during which incorrect authorization results are returned.
Optional.ofNullable(delta.aclsDelta()).ifPresent(aclsDelta -> { Optional.ofNullable(delta.aclsDelta()).ifPresent(aclsDelta -> {
authorizer.ifPresent(clusterMetadataAuthorizer -> { authorizer.ifPresent(authorizer -> {
ClusterMetadataAuthorizer clusterMetadataAuthorizer = (ClusterMetadataAuthorizer) authorizer.get();
if (manifest.type().equals(LoaderManifestType.SNAPSHOT)) { if (manifest.type().equals(LoaderManifestType.SNAPSHOT)) {
try { try {
// If the delta resulted from a snapshot load, we want to apply the new changes // If the delta resulted from a snapshot load, we want to apply the new changes
@ -103,6 +105,9 @@ public class AclPublisher implements MetadataPublisher {
@Override @Override
public void close() { public void close() {
authorizer.ifPresent(clusterMetadataAuthorizer -> clusterMetadataAuthorizer.completeInitialLoad(new TimeoutException())); authorizer.ifPresent(authorizer -> {
ClusterMetadataAuthorizer clusterMetadataAuthorizer = (ClusterMetadataAuthorizer) authorizer.get();
clusterMetadataAuthorizer.completeInitialLoad(new TimeoutException());
});
} }
} }

View File

@ -27,6 +27,8 @@ import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.AuthorizerNotReadyException; import org.apache.kafka.common.errors.AuthorizerNotReadyException;
import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourcePatternFilter;
@ -49,8 +51,6 @@ import org.slf4j.LoggerFactory;
import java.net.InetAddress; import java.net.InetAddress;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -101,12 +101,9 @@ public class StandardAuthorizerTest {
"127.0.0.1", "127.0.0.1",
9020); 9020);
public static class AuthorizerTestServerInfo implements AuthorizerServerInfo { public record AuthorizerTestServerInfo(Collection<Endpoint> endpoints) implements AuthorizerServerInfo {
private final Collection<Endpoint> endpoints; public AuthorizerTestServerInfo {
public AuthorizerTestServerInfo(Collection<Endpoint> endpoints) {
assertFalse(endpoints.isEmpty()); assertFalse(endpoints.isEmpty());
this.endpoints = endpoints;
} }
@Override @Override
@ -119,11 +116,6 @@ public class StandardAuthorizerTest {
return 0; return 0;
} }
@Override
public Collection<Endpoint> endpoints() {
return endpoints;
}
@Override @Override
public Endpoint interBrokerEndpoint() { public Endpoint interBrokerEndpoint() {
return endpoints.iterator().next(); return endpoints.iterator().next();
@ -141,15 +133,17 @@ public class StandardAuthorizerTest {
} }
} }
private final Metrics metrics = new Metrics();
@Test @Test
public void testGetConfiguredSuperUsers() { public void testGetConfiguredSuperUsers() {
assertEquals(Set.of(), assertEquals(Set.of(),
getConfiguredSuperUsers(Map.of())); getConfiguredSuperUsers(Map.of()));
assertEquals(Set.of(), assertEquals(Set.of(),
getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, " "))); getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, " ")));
assertEquals(new HashSet<>(List.of("User:bob", "User:alice")), assertEquals(Set.of("User:bob", "User:alice"),
getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "User:bob;User:alice "))); getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "User:bob;User:alice ")));
assertEquals(new HashSet<>(List.of("User:bob", "User:alice")), assertEquals(Set.of("User:bob", "User:alice"),
getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "; User:bob ; User:alice "))); getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "; User:bob ; User:alice ")));
assertEquals("expected a string in format principalType:principalName but got bob", assertEquals("expected a string in format principalType:principalName but got bob",
assertThrows(IllegalArgumentException.class, () -> getConfiguredSuperUsers( assertThrows(IllegalArgumentException.class, () -> getConfiguredSuperUsers(
@ -167,13 +161,10 @@ public class StandardAuthorizerTest {
@Test @Test
public void testAllowEveryoneIfNoAclFoundConfigEnabled() throws Exception { public void testAllowEveryoneIfNoAclFoundConfigEnabled() throws Exception {
StandardAuthorizer authorizer = new StandardAuthorizer(); Map<String, Object> configs = Map.of(
HashMap<String, Object> configs = new HashMap<>(); SUPER_USERS_CONFIG, "User:alice;User:chris",
configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true");
configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true"); StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(configs);
authorizer.configure(configs);
authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT)));
authorizer.completeInitialLoad();
List<StandardAclWithId> acls = List.of( List<StandardAclWithId> acls = List.of(
withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW)) withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW))
@ -197,13 +188,10 @@ public class StandardAuthorizerTest {
@Test @Test
public void testAllowEveryoneIfNoAclFoundConfigDisabled() throws Exception { public void testAllowEveryoneIfNoAclFoundConfigDisabled() throws Exception {
StandardAuthorizer authorizer = new StandardAuthorizer(); Map<String, Object> configs = Map.of(
HashMap<String, Object> configs = new HashMap<>(); SUPER_USERS_CONFIG, "User:alice;User:chris",
configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "false");
configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "false"); StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(configs);
authorizer.configure(configs);
authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT)));
authorizer.completeInitialLoad();
List<StandardAclWithId> acls = List.of( List<StandardAclWithId> acls = List.of(
withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW)) withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW))
@ -227,31 +215,35 @@ public class StandardAuthorizerTest {
@Test @Test
public void testConfigure() { public void testConfigure() {
StandardAuthorizer authorizer = new StandardAuthorizer(); Map<String, Object> configs = Map.of(
HashMap<String, Object> configs = new HashMap<>(); SUPER_USERS_CONFIG, "User:alice;User:chris",
configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true");
configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true"); StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(configs);
authorizer.configure(configs); assertEquals(Set.of("User:alice", "User:chris"), authorizer.superUsers());
assertEquals(new HashSet<>(List.of("User:alice", "User:chris")), authorizer.superUsers());
assertEquals(ALLOWED, authorizer.defaultResult()); assertEquals(ALLOWED, authorizer.defaultResult());
} }
static Action newAction(AclOperation aclOperation, private static Action newAction(AclOperation aclOperation,
ResourceType resourceType, ResourceType resourceType,
String resourceName) { String resourceName) {
return new Action(aclOperation, return new Action(aclOperation,
new ResourcePattern(resourceType, resourceName, LITERAL), 1, false, false); new ResourcePattern(resourceType, resourceName, LITERAL), 1, false, false);
} }
static StandardAuthorizer createAndInitializeStandardAuthorizer() { private StandardAuthorizer createAndInitializeStandardAuthorizer() {
return createAndInitializeStandardAuthorizer(Map.of(SUPER_USERS_CONFIG, "User:superman"));
}
private StandardAuthorizer createAndInitializeStandardAuthorizer(Map<String, Object> configs) {
StandardAuthorizer authorizer = new StandardAuthorizer(); StandardAuthorizer authorizer = new StandardAuthorizer();
authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); authorizer.configure(configs);
authorizer.withPluginMetrics(new PluginMetricsImpl(metrics, Map.of()));
authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT))); authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT)));
authorizer.completeInitialLoad(); authorizer.completeInitialLoad();
return authorizer; return authorizer;
} }
static StandardAcl newFooAcl(AclOperation op, AclPermissionType permission) { private static StandardAcl newFooAcl(AclOperation op, AclPermissionType permission) {
return new StandardAcl( return new StandardAcl(
TOPIC, TOPIC,
"foo_", "foo_",
@ -262,7 +254,7 @@ public class StandardAuthorizerTest {
permission); permission);
} }
static StandardAclWithId withId(StandardAcl acl) { private static StandardAclWithId withId(StandardAcl acl) {
return new StandardAclWithId(new Uuid(acl.hashCode(), acl.hashCode()), acl); return new StandardAclWithId(new Uuid(acl.hashCode(), acl.hashCode()), acl);
} }
@ -311,7 +303,7 @@ public class StandardAuthorizerTest {
newFooAcl(ALTER_CONFIGS, DENY))); newFooAcl(ALTER_CONFIGS, DENY)));
} }
static StandardAcl newBarAcl(AclOperation op, AclPermissionType permission) { private static StandardAcl newBarAcl(AclOperation op, AclPermissionType permission) {
return new StandardAcl( return new StandardAcl(
GROUP, GROUP,
"bar", "bar",
@ -641,7 +633,7 @@ public class StandardAuthorizerTest {
authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman"));
Map<Endpoint, ? extends CompletionStage<Void>> futures2 = authorizer. Map<Endpoint, ? extends CompletionStage<Void>> futures2 = authorizer.
start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER))); start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER)));
assertEquals(new HashSet<>(List.of(PLAINTEXT, CONTROLLER)), futures2.keySet()); assertEquals(Set.of(PLAINTEXT, CONTROLLER), futures2.keySet());
assertFalse(futures2.get(PLAINTEXT).toCompletableFuture().isDone()); assertFalse(futures2.get(PLAINTEXT).toCompletableFuture().isDone());
assertTrue(futures2.get(CONTROLLER).toCompletableFuture().isDone()); assertTrue(futures2.get(CONTROLLER).toCompletableFuture().isDone());
} }
@ -656,6 +648,7 @@ public class StandardAuthorizerTest {
public void testAuthorizationPriorToCompleteInitialLoad() throws Exception { public void testAuthorizationPriorToCompleteInitialLoad() throws Exception {
StandardAuthorizer authorizer = new StandardAuthorizer(); StandardAuthorizer authorizer = new StandardAuthorizer();
authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman"));
authorizer.withPluginMetrics(new PluginMetricsImpl(new Metrics(), Map.of()));
assertThrows(AuthorizerNotReadyException.class, () -> assertThrows(AuthorizerNotReadyException.class, () ->
authorizer.authorize(new MockAuthorizableRequestContext.Builder(). authorizer.authorize(new MockAuthorizableRequestContext.Builder().
setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(),
@ -687,7 +680,7 @@ public class StandardAuthorizerTest {
authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman"));
Map<Endpoint, ? extends CompletionStage<Void>> futures = authorizer. Map<Endpoint, ? extends CompletionStage<Void>> futures = authorizer.
start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER))); start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER)));
assertEquals(new HashSet<>(List.of(PLAINTEXT, CONTROLLER)), futures.keySet()); assertEquals(Set.of(PLAINTEXT, CONTROLLER), futures.keySet());
assertFalse(futures.get(PLAINTEXT).toCompletableFuture().isDone()); assertFalse(futures.get(PLAINTEXT).toCompletableFuture().isDone());
assertTrue(futures.get(CONTROLLER).toCompletableFuture().isDone()); assertTrue(futures.get(CONTROLLER).toCompletableFuture().isDone());
authorizer.completeInitialLoad(new TimeoutException("timed out")); authorizer.completeInitialLoad(new TimeoutException("timed out"));
@ -723,4 +716,17 @@ public class StandardAuthorizerTest {
newAction(WRITE, TOPIC, "bar"), newAction(WRITE, TOPIC, "bar"),
newAction(READ, TOPIC, "baz")))); newAction(READ, TOPIC, "baz"))));
} }
@Test
public void testAuthorizerMetrics() throws Exception {
// There's always 1 metrics by default, the metrics count
assertEquals(1, metrics.metrics().size());
StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer();
assertEquals(List.of(ALLOWED), authorizer.authorize(
new MockAuthorizableRequestContext.Builder().setPrincipal(new KafkaPrincipal(USER_TYPE, "superman")).build(),
List.of(newAction(READ, TOPIC, "green"))));
// StandardAuthorizer has 4 metrics
assertEquals(5, metrics.metrics().size());
}
} }

View File

@ -18,6 +18,7 @@
package org.apache.kafka.server.network; package org.apache.kafka.server.network;
import org.apache.kafka.common.Endpoint; import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.authorizer.AuthorizerServerInfo; import org.apache.kafka.server.authorizer.AuthorizerServerInfo;
@ -79,17 +80,17 @@ public class EndpointReadyFutures {
/** /**
* Build the EndpointReadyFutures object. * Build the EndpointReadyFutures object.
* *
* @param authorizer The authorizer to use, if any. Will be started. * @param authorizerPlugin The authorizer to use, if any. Will be started.
* @param info Server information to be passed to the authorizer. * @param info Server information to be passed to the authorizer.
* *
* @return The new futures object. * @return The new futures object.
*/ */
public EndpointReadyFutures build( public EndpointReadyFutures build(
Optional<Authorizer> authorizer, Optional<Plugin<Authorizer>> authorizerPlugin,
AuthorizerServerInfo info AuthorizerServerInfo info
) { ) {
if (authorizer.isPresent()) { if (authorizerPlugin.isPresent()) {
return build(authorizer.get().start(info), info); return build(authorizerPlugin.get().get().start(info), info);
} else { } else {
return build(Map.of(), info); return build(Map.of(), info);
} }

View File

@ -16,13 +16,19 @@
*/ */
package org.apache.kafka.security.authorizer; package org.apache.kafka.security.authorizer;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.resource.Resource; import org.apache.kafka.common.resource.Resource;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.authorizer.Authorizer;
import java.util.Map;
public class AuthorizerUtils { public class AuthorizerUtils {
public static Authorizer createAuthorizer(String className) throws ClassNotFoundException { public static Plugin<Authorizer> createAuthorizer(String className, Map<String, Object> configs, Metrics metrics, String key, String role) throws ClassNotFoundException {
return Utils.newInstance(className, Authorizer.class); Authorizer authorizer = Utils.newInstance(className, Authorizer.class);
authorizer.configure(configs);
return Plugin.wrapInstance(authorizer, metrics, key, "role", role);
} }
public static boolean isClusterResource(String name) { public static boolean isClusterResource(String name) {

View File

@ -0,0 +1,135 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.security.authorizer;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.server.ProcessRole;
import org.apache.kafka.server.authorizer.AclCreateResult;
import org.apache.kafka.server.authorizer.AclDeleteResult;
import org.apache.kafka.server.authorizer.Action;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import org.apache.kafka.server.authorizer.AuthorizationResult;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.authorizer.AuthorizerServerInfo;
import org.apache.kafka.server.config.ServerConfigs;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class AuthorizerUtilsTest {
@ParameterizedTest
@EnumSource(ProcessRole.class)
public void testCreateAuthorizer(ProcessRole role) throws ClassNotFoundException {
Map<String, Object> configs = Map.of();
Metrics metrics = new Metrics();
assertEquals(1, metrics.metrics().size());
Plugin<Authorizer> authorizer = AuthorizerUtils.createAuthorizer(
MonitorableAuthorizer.class.getName(),
configs,
metrics,
ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG,
role.toString());
assertInstanceOf(MonitorableAuthorizer.class, authorizer.get());
MonitorableAuthorizer monitorableAuthorizer = (MonitorableAuthorizer) authorizer.get();
assertTrue(monitorableAuthorizer.configured);
assertNotNull(monitorableAuthorizer.metricName);
MetricName metricName = null;
for (MetricName name : metrics.metrics().keySet()) {
if (name.name().equals(monitorableAuthorizer.metricName.name())) {
metricName = name;
}
}
assertNotNull(metricName);
assertEquals(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, metricName.tags().get("config"));
assertEquals(MonitorableAuthorizer.class.getSimpleName(), metricName.tags().get("class"));
assertEquals(role.toString(), metricName.tags().get("role"));
assertTrue(metricName.tags().entrySet().containsAll(monitorableAuthorizer.extraTags.entrySet()));
assertEquals(0, metrics.metric(metricName).metricValue());
monitorableAuthorizer.authorize(null, null);
assertEquals(1, metrics.metric(metricName).metricValue());
}
public static class MonitorableAuthorizer implements Authorizer, Monitorable {
private final Map<String, String> extraTags = Map.of("k1", "v1");
private final AtomicInteger counter = new AtomicInteger();
private boolean configured = false;
private MetricName metricName = null;
@Override
public void withPluginMetrics(PluginMetrics metrics) {
assertTrue(configured);
metricName = metrics.metricName("authorize calls", "Number of times authorize was called", extraTags);
metrics.addMetric(metricName, (Gauge<Integer>) (config, now) -> counter.get());
}
@Override
public Map<Endpoint, ? extends CompletionStage<Void>> start(AuthorizerServerInfo serverInfo) {
return Map.of();
}
@Override
public List<AuthorizationResult> authorize(AuthorizableRequestContext requestContext, List<Action> actions) {
counter.incrementAndGet();
return List.of();
}
@Override
public List<? extends CompletionStage<AclCreateResult>> createAcls(AuthorizableRequestContext requestContext, List<AclBinding> aclBindings) {
return List.of();
}
@Override
public List<? extends CompletionStage<AclDeleteResult>> deleteAcls(AuthorizableRequestContext requestContext, List<AclBindingFilter> aclBindingFilters) {
return List.of();
}
@Override
public Iterable<AclBinding> acls(AclBindingFilter filter) {
return null;
}
@Override
public void close() {
}
@Override
public void configure(Map<String, ?> configs) {
configured = true;
}
}
}

View File

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.server;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.replica.RackAwareReplicaSelector;
import org.apache.kafka.common.test.ClusterInstance;
import org.apache.kafka.common.test.api.ClusterConfigProperty;
import org.apache.kafka.common.test.api.ClusterTest;
import org.apache.kafka.common.test.api.Type;
import org.apache.kafka.metadata.authorizer.StandardAuthorizer;
import java.util.LinkedHashMap;
import java.util.Map;
import static org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG;
import static org.apache.kafka.server.config.ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class MonitorablePluginsIntegrationTest {
private static int controllerId(Type type) {
return type == Type.KRAFT ? 3000 : 0;
}
private static Map<String, String> expectedTags(String config, String clazz) {
return expectedTags(config, clazz, Map.of());
}
private static Map<String, String> expectedTags(String config, String clazz, Map<String, String> extraTags) {
Map<String, String> tags = new LinkedHashMap<>();
tags.put("config", config);
tags.put("class", clazz);
tags.putAll(extraTags);
return tags;
}
@ClusterTest(
types = {Type.KRAFT, Type.CO_KRAFT},
serverProperties = {
@ClusterConfigProperty(key = StandardAuthorizer.SUPER_USERS_CONFIG, value = "User:ANONYMOUS"),
@ClusterConfigProperty(key = AUTHORIZER_CLASS_NAME_CONFIG, value = "org.apache.kafka.metadata.authorizer.StandardAuthorizer"),
@ClusterConfigProperty(key = REPLICA_SELECTOR_CLASS_CONFIG, value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableReplicaSelector")
}
)
public void testMonitorableServerPlugins(ClusterInstance clusterInstance) {
assertAuthorizerMetrics(clusterInstance);
assertReplicaSelectorMetrics(clusterInstance);
}
private void assertAuthorizerMetrics(ClusterInstance clusterInstance) {
assertMetrics(
clusterInstance.brokers().get(0).metrics(),
4,
expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "broker")));
assertMetrics(
clusterInstance.controllers().get(controllerId(clusterInstance.type())).metrics(),
4,
expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "controller")));
}
private void assertReplicaSelectorMetrics(ClusterInstance clusterInstance) {
assertMetrics(
clusterInstance.brokers().get(0).metrics(),
MonitorableReplicaSelector.METRICS_COUNT,
expectedTags(REPLICA_SELECTOR_CLASS_CONFIG, MonitorableReplicaSelector.class.getSimpleName()));
}
private void assertMetrics(Metrics metrics, int expected, Map<String, String> expectedTags) {
int found = 0;
for (MetricName metricName : metrics.metrics().keySet()) {
if (metricName.group().equals("plugins")) {
Map<String, String> tags = metricName.tags();
if (expectedTags.equals(tags)) {
found++;
}
}
}
assertEquals(expected, found);
}
public static class MonitorableReplicaSelector extends RackAwareReplicaSelector implements Monitorable {
private static final int METRICS_COUNT = 1;
@Override
public void withPluginMetrics(PluginMetrics metrics) {
MetricName name = metrics.metricName("name", "description", Map.of());
metrics.addMetric(name, (Measurable) (config, now) -> 123);
}
}
}

View File

@ -357,11 +357,11 @@ public interface ClusterInstance {
default List<Authorizer> authorizers() { default List<Authorizer> authorizers() {
List<Authorizer> authorizers = new ArrayList<>(); List<Authorizer> authorizers = new ArrayList<>();
authorizers.addAll(brokers().values().stream() authorizers.addAll(brokers().values().stream()
.filter(server -> server.authorizer().isDefined()) .filter(server -> server.authorizerPlugin().isDefined())
.map(server -> server.authorizer().get()).toList()); .map(server -> server.authorizerPlugin().get().get()).toList());
authorizers.addAll(controllers().values().stream() authorizers.addAll(controllers().values().stream()
.filter(server -> server.authorizer().isDefined()) .filter(server -> server.authorizerPlugin().isDefined())
.map(server -> server.authorizer().get()).toList()); .map(server -> server.authorizerPlugin().get().get()).toList());
return authorizers; return authorizers;
} }