KAFKA-8847; Deprecate and remove usage of supporting classes in kafka.security.auth (#7966)

Removes references to the old scala Acl classes from kafka.security.auth (Acl, Operation, ResourceType, Resource etc.) and replaces these with the Java API. Only the old SimpleAclAuthorizer, AuthorizerWrapper used to wrap legacy authorizer instances and tests using SimpleAclAuthorizer continue to use the old API. Deprecates the old scala API.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
This commit is contained in:
Rajini Sivaram 2020-01-17 13:20:34 +00:00 committed by GitHub
parent 08e1576eee
commit dbeaba5d9e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 688 additions and 547 deletions

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.utils; package org.apache.kafka.common.utils;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.config.SecurityConfig;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
import org.apache.kafka.common.security.auth.SecurityProviderCreator; import org.apache.kafka.common.security.auth.SecurityProviderCreator;
@ -26,6 +27,7 @@ import org.slf4j.LoggerFactory;
import java.security.Security; import java.security.Security;
import java.util.HashMap; import java.util.HashMap;
import java.util.Locale;
import java.util.Map; import java.util.Map;
public class SecurityUtils { public class SecurityUtils {
@ -34,18 +36,27 @@ public class SecurityUtils {
private static final Map<String, ResourceType> NAME_TO_RESOURCE_TYPES; private static final Map<String, ResourceType> NAME_TO_RESOURCE_TYPES;
private static final Map<String, AclOperation> NAME_TO_OPERATIONS; private static final Map<String, AclOperation> NAME_TO_OPERATIONS;
private static final Map<String, AclPermissionType> NAME_TO_PERMISSION_TYPES;
static { static {
NAME_TO_RESOURCE_TYPES = new HashMap<>(ResourceType.values().length); NAME_TO_RESOURCE_TYPES = new HashMap<>(ResourceType.values().length);
NAME_TO_OPERATIONS = new HashMap<>(AclOperation.values().length); NAME_TO_OPERATIONS = new HashMap<>(AclOperation.values().length);
NAME_TO_PERMISSION_TYPES = new HashMap<>(AclPermissionType.values().length);
for (ResourceType resourceType : ResourceType.values()) { for (ResourceType resourceType : ResourceType.values()) {
String resourceTypeName = toPascalCase(resourceType.name()); String resourceTypeName = toPascalCase(resourceType.name());
NAME_TO_RESOURCE_TYPES.put(resourceTypeName, resourceType); NAME_TO_RESOURCE_TYPES.put(resourceTypeName, resourceType);
NAME_TO_RESOURCE_TYPES.put(resourceTypeName.toUpperCase(Locale.ROOT), resourceType);
} }
for (AclOperation operation : AclOperation.values()) { for (AclOperation operation : AclOperation.values()) {
String operationName = toPascalCase(operation.name()); String operationName = toPascalCase(operation.name());
NAME_TO_OPERATIONS.put(operationName, operation); NAME_TO_OPERATIONS.put(operationName, operation);
NAME_TO_OPERATIONS.put(operationName.toUpperCase(Locale.ROOT), operation);
}
for (AclPermissionType permissionType : AclPermissionType.values()) {
String permissionName = toPascalCase(permissionType.name());
NAME_TO_PERMISSION_TYPES.put(permissionName, permissionType);
NAME_TO_PERMISSION_TYPES.put(permissionName.toUpperCase(Locale.ROOT), permissionType);
} }
} }
@ -86,13 +97,38 @@ public class SecurityUtils {
} }
public static ResourceType resourceType(String name) { public static ResourceType resourceType(String name) {
ResourceType resourceType = NAME_TO_RESOURCE_TYPES.get(name); return valueFromMap(NAME_TO_RESOURCE_TYPES, name, ResourceType.UNKNOWN);
return resourceType == null ? ResourceType.UNKNOWN : resourceType;
} }
public static AclOperation operation(String name) { public static AclOperation operation(String name) {
AclOperation operation = NAME_TO_OPERATIONS.get(name); return valueFromMap(NAME_TO_OPERATIONS, name, AclOperation.UNKNOWN);
return operation == null ? AclOperation.UNKNOWN : operation; }
public static AclPermissionType permissionType(String name) {
return valueFromMap(NAME_TO_PERMISSION_TYPES, name, AclPermissionType.UNKNOWN);
}
// We use Pascal-case to store these values, so lookup using provided key first to avoid
// case conversion for the common case. For backward compatibility, also perform
// case-insensitive look up (without underscores) by converting the key to upper-case.
private static <T> T valueFromMap(Map<String, T> map, String key, T unknown) {
T value = map.get(key);
if (value == null) {
value = map.get(key.toUpperCase(Locale.ROOT));
}
return value == null ? unknown : value;
}
public static String resourceTypeName(ResourceType resourceType) {
return toPascalCase(resourceType.name());
}
public static String operationName(AclOperation operation) {
return toPascalCase(operation.name());
}
public static String permissionTypeName(AclPermissionType permissionType) {
return toPascalCase(permissionType.name());
} }
private static String toPascalCase(String name) { private static String toPascalCase(String name) {

View File

@ -21,8 +21,7 @@ import java.util.Properties
import joptsimple._ import joptsimple._
import joptsimple.util.EnumConverter import joptsimple.util.EnumConverter
import kafka.security.auth._ import kafka.security.authorizer.{AclAuthorizer, AclEntry, AuthorizerUtils}
import kafka.security.authorizer.AuthorizerUtils
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils._ import kafka.utils._
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
@ -33,7 +32,7 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceP
import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.{Utils, SecurityUtils => JSecurityUtils} import org.apache.kafka.common.utils.{Utils, SecurityUtils => JSecurityUtils}
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.Authorizer
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
@ -58,18 +57,12 @@ object AclCommand extends Logging {
if (opts.options.has(opts.bootstrapServerOpt)) { if (opts.options.has(opts.bootstrapServerOpt)) {
new AdminClientService(opts) new AdminClientService(opts)
} else { } else {
val authorizerClass = if (opts.options.has(opts.authorizerOpt)) { val authorizerClassName = if (opts.options.has(opts.authorizerOpt))
val className = opts.options.valueOf(opts.authorizerOpt) opts.options.valueOf(opts.authorizerOpt)
Class.forName(className, true, Utils.getContextOrKafkaClassLoader)
} else
classOf[SimpleAclAuthorizer]
if (classOf[JAuthorizer].isAssignableFrom(authorizerClass))
new JAuthorizerService(authorizerClass.asSubclass(classOf[JAuthorizer]), opts)
else if (classOf[Authorizer].isAssignableFrom(authorizerClass))
new AuthorizerService(authorizerClass.asSubclass(classOf[Authorizer]), opts)
else else
throw new IllegalArgumentException(s"Authorizer $authorizerClass does not implement ${classOf[Authorizer]} or ${classOf[JAuthorizer]}.") classOf[AclAuthorizer].getName
new AuthorizerService(authorizerClassName, opts)
} }
} }
@ -190,8 +183,7 @@ object AclCommand extends Logging {
} }
} }
@deprecated("Use JAuthorizerService", "Since 2.4") class AuthorizerService(val authorizerClassName: String, val opts: AclCommandOptions) extends AclCommandService with Logging {
class AuthorizerService(val authorizerClass: Class[_ <: Authorizer], val opts: AclCommandOptions) extends AclCommandService with Logging {
private def withAuthorizer()(f: Authorizer => Unit): Unit = { private def withAuthorizer()(f: Authorizer => Unit): Unit = {
val defaultProps = Map(KafkaConfig.ZkEnableSecureAclsProp -> JaasUtils.isZkSecurityEnabled) val defaultProps = Map(KafkaConfig.ZkEnableSecureAclsProp -> JaasUtils.isZkSecurityEnabled)
@ -203,105 +195,7 @@ object AclCommand extends Logging {
defaultProps defaultProps
} }
val authZ = Utils.newInstance(authorizerClass) val authZ = AuthorizerUtils.createAuthorizer(authorizerClassName)
try {
authZ.configure(authorizerProperties.asJava)
f(authZ)
}
finally CoreUtils.swallow(authZ.close(), this)
}
def addAcls(): Unit = {
val resourceToAcl = getResourceToAcls(opts)
withAuthorizer() { authorizer =>
for ((resource, acls) <- resourceToAcl) {
println(s"Adding ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
authorizer.addAcls(acls.map(AuthorizerUtils.convertToAcl), AuthorizerUtils.convertToResource(resource))
}
listAcls()
}
}
def removeAcls(): Unit = {
withAuthorizer() { authorizer =>
val filterToAcl = getResourceFilterToAcls(opts)
for ((filter, acls) <- filterToAcl) {
if (acls.isEmpty) {
if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource filter `$filter`? (y/n)"))
removeAcls(authorizer, acls, filter)
} else {
if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource filter `$filter`? (y/n)"))
removeAcls(authorizer, acls, filter)
}
}
listAcls()
}
}
def listAcls(): Unit = {
withAuthorizer() { authorizer =>
val filters = getResourceFilter(opts, dieIfNoResourceFound = false)
val listPrincipals = getPrincipals(opts, opts.listPrincipalsOpt)
if (listPrincipals.isEmpty) {
val resourceToAcls = getFilteredResourceToAcls(authorizer, filters)
for ((resource, acls) <- resourceToAcls)
println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
} else {
listPrincipals.foreach(principal => {
println(s"ACLs for principal `$principal`")
val resourceToAcls = getFilteredResourceToAcls(authorizer, filters, Some(principal))
for ((resource, acls) <- resourceToAcls)
println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
})
}
}
}
private def removeAcls(authorizer: Authorizer, acls: Set[AccessControlEntry], filter: ResourcePatternFilter): Unit = {
getAcls(authorizer, filter)
.keys
.foreach(resource =>
if (acls.isEmpty) authorizer.removeAcls(resource)
else authorizer.removeAcls(acls.map(AuthorizerUtils.convertToAcl), resource)
)
}
private def getFilteredResourceToAcls(authorizer: Authorizer, filters: Set[ResourcePatternFilter],
listPrincipal: Option[KafkaPrincipal] = None): Iterable[(Resource, Set[Acl])] = {
if (filters.isEmpty)
if (listPrincipal.isEmpty)
authorizer.getAcls()
else
authorizer.getAcls(listPrincipal.get)
else filters.flatMap(filter => getAcls(authorizer, filter, listPrincipal))
}
private def getAcls(authorizer: Authorizer, filter: ResourcePatternFilter,
listPrincipal: Option[KafkaPrincipal] = None): Map[Resource, Set[Acl]] =
if (listPrincipal.isEmpty)
authorizer.getAcls().filter { case (resource, _) => filter.matches(resource.toPattern) }
else
authorizer.getAcls(listPrincipal.get).filter { case (resource, _) => filter.matches(resource.toPattern) }
}
class JAuthorizerService(val authorizerClass: Class[_ <: JAuthorizer], val opts: AclCommandOptions) extends AclCommandService with Logging {
private def withAuthorizer()(f: JAuthorizer => Unit): Unit = {
val defaultProps = Map(KafkaConfig.ZkEnableSecureAclsProp -> JaasUtils.isZkSecurityEnabled)
val authorizerProperties =
if (opts.options.has(opts.authorizerPropertiesOpt)) {
val authorizerProperties = opts.options.valuesOf(opts.authorizerPropertiesOpt).asScala
defaultProps ++ CommandLineUtils.parseKeyValueArgs(authorizerProperties, acceptMissingValue = false).asScala
} else {
defaultProps
}
val authZ = Utils.newInstance(authorizerClass)
try { try {
authZ.configure(authorizerProperties.asJava) authZ.configure(authorizerProperties.asJava)
f(authZ) f(authZ)
@ -367,7 +261,7 @@ object AclCommand extends Logging {
} }
} }
private def removeAcls(authorizer: JAuthorizer, acls: Set[AccessControlEntry], filter: ResourcePatternFilter): Unit = { private def removeAcls(authorizer: Authorizer, acls: Set[AccessControlEntry], filter: ResourcePatternFilter): Unit = {
val result = if (acls.isEmpty) val result = if (acls.isEmpty)
authorizer.deleteAcls(null, List(new AclBindingFilter(filter, AccessControlEntryFilter.ANY)).asJava) authorizer.deleteAcls(null, List(new AclBindingFilter(filter, AccessControlEntryFilter.ANY)).asJava)
else { else {
@ -388,7 +282,7 @@ object AclCommand extends Logging {
} }
} }
private def getAcls(authorizer: JAuthorizer, filters: Set[ResourcePatternFilter]): Map[ResourcePattern, Set[AccessControlEntry]] = { private def getAcls(authorizer: Authorizer, filters: Set[ResourcePatternFilter]): Map[ResourcePattern, Set[AccessControlEntry]] = {
val aclBindings = val aclBindings =
if (filters.isEmpty) authorizer.acls(AclBindingFilter.ANY).asScala if (filters.isEmpty) authorizer.acls(AclBindingFilter.ANY).asScala
else { else {
@ -500,7 +394,8 @@ object AclCommand extends Logging {
} }
private def getAcl(opts: AclCommandOptions): Set[AccessControlEntry] = { private def getAcl(opts: AclCommandOptions): Set[AccessControlEntry] = {
val operations = opts.options.valuesOf(opts.operationsOpt).asScala.map(operation => Operation.fromString(operation.trim)).map(_.toJava).toSet val operations = opts.options.valuesOf(opts.operationsOpt).asScala
.map(operation => JSecurityUtils.operation(operation.trim)).toSet
getAcl(opts, operations) getAcl(opts, operations)
} }
@ -518,7 +413,7 @@ object AclCommand extends Logging {
if (opts.options.has(hostOptionSpec)) if (opts.options.has(hostOptionSpec))
opts.options.valuesOf(hostOptionSpec).asScala.map(_.trim).toSet opts.options.valuesOf(hostOptionSpec).asScala.map(_.trim).toSet
else if (opts.options.has(principalOptionSpec)) else if (opts.options.has(principalOptionSpec))
Set[String](Acl.WildCardHost) Set[String](AclEntry.WildcardHost)
else else
Set.empty[String] Set.empty[String]
} }
@ -565,8 +460,8 @@ object AclCommand extends Logging {
private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[ResourcePatternFilter, Set[AccessControlEntry]]): Unit = { private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[ResourcePatternFilter, Set[AccessControlEntry]]): Unit = {
for ((resource, acls) <- resourceToAcls) { for ((resource, acls) <- resourceToAcls) {
val validOps = ResourceType.fromJava(resource.resourceType).supportedOperations + All val validOps = AclEntry.supportedOperations(resource.resourceType) + AclOperation.ALL
if ((acls.map(_.operation) -- validOps.map(_.toJava)).nonEmpty) if ((acls.map(_.operation) -- validOps).nonEmpty)
CommandLineUtils.printUsageAndDie(opts.parser, s"ResourceType ${resource.resourceType} only supports operations ${validOps.mkString(",")}") CommandLineUtils.printUsageAndDie(opts.parser, s"ResourceType ${resource.resourceType} only supports operations ${validOps.mkString(",")}")
} }
} }
@ -641,10 +536,10 @@ object AclCommand extends Logging {
val listOpt = parser.accepts("list", "List ACLs for the specified resource, use --topic <topic> or --group <group> or --cluster to specify a resource.") val listOpt = parser.accepts("list", "List ACLs for the specified resource, use --topic <topic> or --group <group> or --cluster to specify a resource.")
val operationsOpt = parser.accepts("operation", "Operation that is being allowed or denied. Valid operation names are: " + Newline + val operationsOpt = parser.accepts("operation", "Operation that is being allowed or denied. Valid operation names are: " + Newline +
Operation.values.map("\t" + _).mkString(Newline) + Newline) AclEntry.AclOperations.map("\t" + JSecurityUtils.operationName(_)).mkString(Newline) + Newline)
.withRequiredArg .withRequiredArg
.ofType(classOf[String]) .ofType(classOf[String])
.defaultsTo(All.name) .defaultsTo(JSecurityUtils.operationName(AclOperation.ALL))
val allowPrincipalsOpt = parser.accepts("allow-principal", "principal is in principalType:name format." + val allowPrincipalsOpt = parser.accepts("allow-principal", "principal is in principalType:name format." +
" Note that principalType must be supported by the Authorizer being used." + " Note that principalType must be supported by the Authorizer being used." +

View File

@ -17,65 +17,40 @@
package kafka.security.auth package kafka.security.auth
import kafka.utils.Json import kafka.security.authorizer.AclEntry
import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.SecurityUtils
import scala.collection.JavaConverters._
@deprecated("Use org.apache.kafka.common.acl.AclBinding", "Since 2.5")
object Acl { object Acl {
val WildCardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*") val WildCardPrincipal: KafkaPrincipal = AclEntry.WildcardPrincipal
val WildCardHost: String = "*" val WildCardHost: String = AclEntry.WildcardHost
val WildCardResource: String = ResourcePattern.WILDCARD_RESOURCE val WildCardResource: String = ResourcePattern.WILDCARD_RESOURCE
val AllowAllAcl = new Acl(WildCardPrincipal, Allow, WildCardHost, All) val AllowAllAcl = new Acl(WildCardPrincipal, Allow, WildCardHost, All)
val PrincipalKey = "principal" val PrincipalKey = AclEntry.PrincipalKey
val PermissionTypeKey = "permissionType" val PermissionTypeKey = AclEntry.PermissionTypeKey
val OperationKey = "operation" val OperationKey = AclEntry.OperationKey
val HostsKey = "host" val HostsKey = AclEntry.HostsKey
val VersionKey = "version" val VersionKey = AclEntry.VersionKey
val CurrentVersion = 1 val CurrentVersion = AclEntry.CurrentVersion
val AclsKey = "acls" val AclsKey = AclEntry.AclsKey
/** /**
* *
* @param bytes of acls json string * @see AclEntry
*
* <p>
{
"version": 1,
"acls": [
{
"host":"host1",
"permissionType": "Deny",
"operation": "Read",
"principal": "User:alice"
}
]
}
* </p>
*
* @return
*/ */
def fromBytes(bytes: Array[Byte]): Set[Acl] = { def fromBytes(bytes: Array[Byte]): Set[Acl] = {
if (bytes == null || bytes.isEmpty) AclEntry.fromBytes(bytes)
return collection.immutable.Set.empty[Acl] .map(ace => Acl(ace.kafkaPrincipal,
PermissionType.fromJava(ace.permissionType()),
Json.parseBytes(bytes).map(_.asJsonObject).map { js => ace.host(),
//the acl json version. Operation.fromJava(ace.operation())))
require(js(VersionKey).to[Int] == CurrentVersion)
js(AclsKey).asJsonArray.iterator.map(_.asJsonObject).map { itemJs =>
val principal = SecurityUtils.parseKafkaPrincipal(itemJs(PrincipalKey).to[String])
val permissionType = PermissionType.fromString(itemJs(PermissionTypeKey).to[String])
val host = itemJs(HostsKey).to[String]
val operation = Operation.fromString(itemJs(OperationKey).to[String])
new Acl(principal, permissionType, host, operation)
}.toSet
}.getOrElse(Set.empty)
} }
def toJsonCompatibleMap(acls: Set[Acl]): Map[String, Any] = { def toJsonCompatibleMap(acls: Set[Acl]): Map[String, Any] = {
Map(Acl.VersionKey -> Acl.CurrentVersion, Acl.AclsKey -> acls.map(acl => acl.toMap.asJava).toList.asJava) AclEntry.toJsonCompatibleMap(acls.map(acl =>
AclEntry(acl.principal, acl.permissionType.toJava, acl.host, acl.operation.toJava)
))
} }
} }
@ -89,6 +64,7 @@ object Acl {
* @param host A value of * indicates all hosts. * @param host A value of * indicates all hosts.
* @param operation A value of ALL indicates all operations. * @param operation A value of ALL indicates all operations.
*/ */
@deprecated("Use org.apache.kafka.common.acl.AclBinding", "Since 2.5")
case class Acl(principal: KafkaPrincipal, permissionType: PermissionType, host: String, operation: Operation) { case class Acl(principal: KafkaPrincipal, permissionType: PermissionType, host: String, operation: Operation) {
/** /**

View File

@ -22,56 +22,68 @@ import org.apache.kafka.common.acl.AclOperation
/** /**
* Different operations a client may perform on kafka resources. * Different operations a client may perform on kafka resources.
*/ */
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
sealed trait Operation extends BaseEnum { sealed trait Operation extends BaseEnum {
def toJava : AclOperation def toJava : AclOperation
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Read extends Operation { case object Read extends Operation {
val name = "Read" val name = "Read"
val toJava = AclOperation.READ val toJava = AclOperation.READ
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Write extends Operation { case object Write extends Operation {
val name = "Write" val name = "Write"
val toJava = AclOperation.WRITE val toJava = AclOperation.WRITE
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Create extends Operation { case object Create extends Operation {
val name = "Create" val name = "Create"
val toJava = AclOperation.CREATE val toJava = AclOperation.CREATE
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Delete extends Operation { case object Delete extends Operation {
val name = "Delete" val name = "Delete"
val toJava = AclOperation.DELETE val toJava = AclOperation.DELETE
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Alter extends Operation { case object Alter extends Operation {
val name = "Alter" val name = "Alter"
val toJava = AclOperation.ALTER val toJava = AclOperation.ALTER
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object Describe extends Operation { case object Describe extends Operation {
val name = "Describe" val name = "Describe"
val toJava = AclOperation.DESCRIBE val toJava = AclOperation.DESCRIBE
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object ClusterAction extends Operation { case object ClusterAction extends Operation {
val name = "ClusterAction" val name = "ClusterAction"
val toJava = AclOperation.CLUSTER_ACTION val toJava = AclOperation.CLUSTER_ACTION
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object DescribeConfigs extends Operation { case object DescribeConfigs extends Operation {
val name = "DescribeConfigs" val name = "DescribeConfigs"
val toJava = AclOperation.DESCRIBE_CONFIGS val toJava = AclOperation.DESCRIBE_CONFIGS
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object AlterConfigs extends Operation { case object AlterConfigs extends Operation {
val name = "AlterConfigs" val name = "AlterConfigs"
val toJava = AclOperation.ALTER_CONFIGS val toJava = AclOperation.ALTER_CONFIGS
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object IdempotentWrite extends Operation { case object IdempotentWrite extends Operation {
val name = "IdempotentWrite" val name = "IdempotentWrite"
val toJava = AclOperation.IDEMPOTENT_WRITE val toJava = AclOperation.IDEMPOTENT_WRITE
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
case object All extends Operation { case object All extends Operation {
val name = "All" val name = "All"
val toJava = AclOperation.ALL val toJava = AclOperation.ALL
} }
@deprecated("Use org.apache.kafka.common.acl.AclOperation", "Since 2.5")
object Operation { object Operation {
def fromString(operation: String): Operation = { def fromString(operation: String): Operation = {

View File

@ -19,20 +19,24 @@ package kafka.security.auth
import kafka.common.{BaseEnum, KafkaException} import kafka.common.{BaseEnum, KafkaException}
import org.apache.kafka.common.acl.AclPermissionType import org.apache.kafka.common.acl.AclPermissionType
@deprecated("Use org.apache.kafka.common.acl.AclPermissionType", "Since 2.5")
sealed trait PermissionType extends BaseEnum { sealed trait PermissionType extends BaseEnum {
val toJava: AclPermissionType val toJava: AclPermissionType
} }
@deprecated("Use org.apache.kafka.common.acl.AclPermissionType", "Since 2.5")
case object Allow extends PermissionType { case object Allow extends PermissionType {
val name = "Allow" val name = "Allow"
val toJava = AclPermissionType.ALLOW val toJava = AclPermissionType.ALLOW
} }
@deprecated("Use org.apache.kafka.common.acl.AclPermissionType", "Since 2.5")
case object Deny extends PermissionType { case object Deny extends PermissionType {
val name = "Deny" val name = "Deny"
val toJava = AclPermissionType.DENY val toJava = AclPermissionType.DENY
} }
@deprecated("Use org.apache.kafka.common.acl.AclPermissionType", "Since 2.5")
object PermissionType { object PermissionType {
def fromString(permissionType: String): PermissionType = { def fromString(permissionType: String): PermissionType = {
val pType = values.find(pType => pType.name.equalsIgnoreCase(permissionType)) val pType = values.find(pType => pType.name.equalsIgnoreCase(permissionType))

View File

@ -17,13 +17,15 @@
package kafka.security.auth package kafka.security.auth
import kafka.common.KafkaException import kafka.common.KafkaException
import kafka.security.authorizer.AclEntry
import org.apache.kafka.common.resource.{PatternType, ResourcePattern} import org.apache.kafka.common.resource.{PatternType, ResourcePattern}
@deprecated("Use org.apache.kafka.common.resource.ResourcePattern", "Since 2.5")
object Resource { object Resource {
val Separator = ":" val Separator = AclEntry.ResourceSeparator
val ClusterResourceName = "kafka-cluster" val ClusterResourceName = "kafka-cluster"
val ClusterResource = Resource(Cluster, Resource.ClusterResourceName, PatternType.LITERAL) val ClusterResource = Resource(Cluster, Resource.ClusterResourceName, PatternType.LITERAL)
val WildCardResource = "*" val WildCardResource = AclEntry.WildcardResource
@deprecated("This resource name is not used by Kafka and will be removed in a future release", since = "2.1") @deprecated("This resource name is not used by Kafka and will be removed in a future release", since = "2.1")
val ProducerIdResourceName = "producer-id" // This is not used since we don't have a producer id resource val ProducerIdResourceName = "producer-id" // This is not used since we don't have a producer id resource
@ -53,6 +55,7 @@ object Resource {
* it will be a constant string kafka-cluster. * it will be a constant string kafka-cluster.
* @param patternType non-null resource pattern type: literal, prefixed, etc. * @param patternType non-null resource pattern type: literal, prefixed, etc.
*/ */
@deprecated("Use org.apache.kafka.common.resource.ResourcePattern", "Since 2.5")
case class Resource(resourceType: ResourceType, name: String, patternType: PatternType) { case class Resource(resourceType: ResourceType, name: String, patternType: PatternType) {
if (!patternType.isSpecific) if (!patternType.isSpecific)

View File

@ -20,6 +20,7 @@ import kafka.common.{BaseEnum, KafkaException}
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.resource.{ResourceType => JResourceType} import org.apache.kafka.common.resource.{ResourceType => JResourceType}
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
sealed trait ResourceType extends BaseEnum with Ordered[ ResourceType ] { sealed trait ResourceType extends BaseEnum with Ordered[ ResourceType ] {
def error: Errors def error: Errors
def toJava: JResourceType def toJava: JResourceType
@ -29,6 +30,7 @@ sealed trait ResourceType extends BaseEnum with Ordered[ ResourceType ] {
override def compare(that: ResourceType): Int = this.name compare that.name override def compare(that: ResourceType): Int = this.name compare that.name
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
case object Topic extends ResourceType { case object Topic extends ResourceType {
val name = "Topic" val name = "Topic"
val error = Errors.TOPIC_AUTHORIZATION_FAILED val error = Errors.TOPIC_AUTHORIZATION_FAILED
@ -36,6 +38,7 @@ case object Topic extends ResourceType {
val supportedOperations = Set(Read, Write, Create, Describe, Delete, Alter, DescribeConfigs, AlterConfigs) val supportedOperations = Set(Read, Write, Create, Describe, Delete, Alter, DescribeConfigs, AlterConfigs)
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
case object Group extends ResourceType { case object Group extends ResourceType {
val name = "Group" val name = "Group"
val error = Errors.GROUP_AUTHORIZATION_FAILED val error = Errors.GROUP_AUTHORIZATION_FAILED
@ -43,6 +46,7 @@ case object Group extends ResourceType {
val supportedOperations = Set(Read, Describe, Delete) val supportedOperations = Set(Read, Describe, Delete)
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
case object Cluster extends ResourceType { case object Cluster extends ResourceType {
val name = "Cluster" val name = "Cluster"
val error = Errors.CLUSTER_AUTHORIZATION_FAILED val error = Errors.CLUSTER_AUTHORIZATION_FAILED
@ -50,6 +54,7 @@ case object Cluster extends ResourceType {
val supportedOperations = Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, Alter, Describe) val supportedOperations = Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, Alter, Describe)
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
case object TransactionalId extends ResourceType { case object TransactionalId extends ResourceType {
val name = "TransactionalId" val name = "TransactionalId"
val error = Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED val error = Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED
@ -57,6 +62,7 @@ case object TransactionalId extends ResourceType {
val supportedOperations = Set(Describe, Write) val supportedOperations = Set(Describe, Write)
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
case object DelegationToken extends ResourceType { case object DelegationToken extends ResourceType {
val name = "DelegationToken" val name = "DelegationToken"
val error = Errors.DELEGATION_TOKEN_AUTHORIZATION_FAILED val error = Errors.DELEGATION_TOKEN_AUTHORIZATION_FAILED
@ -64,6 +70,7 @@ case object DelegationToken extends ResourceType {
val supportedOperations : Set[Operation] = Set(Describe) val supportedOperations : Set[Operation] = Set(Describe)
} }
@deprecated("Use org.apache.kafka.common.resource.ResourceType", "Since 2.5")
object ResourceType { object ResourceType {
def fromString(resourceType: String): ResourceType = { def fromString(resourceType: String): ResourceType = {

View File

@ -20,7 +20,7 @@ import java.util
import kafka.network.RequestChannel.Session import kafka.network.RequestChannel.Session
import kafka.security.auth.SimpleAclAuthorizer.BaseAuthorizer import kafka.security.auth.SimpleAclAuthorizer.BaseAuthorizer
import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils} import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils, AuthorizerWrapper}
import kafka.utils._ import kafka.utils._
import kafka.zk.ZkVersion import kafka.zk.ZkVersion
import org.apache.kafka.common.acl.{AccessControlEntryFilter, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} import org.apache.kafka.common.acl.{AccessControlEntryFilter, AclBinding, AclBindingFilter, AclOperation, AclPermissionType}
@ -57,7 +57,7 @@ object SimpleAclAuthorizer {
val principal = requestContext.principal val principal = requestContext.principal
val host = requestContext.clientAddress.getHostAddress val host = requestContext.clientAddress.getHostAddress
val operation = Operation.fromJava(action.operation) val operation = Operation.fromJava(action.operation)
val resource = AuthorizerUtils.convertToResource(action.resourcePattern) val resource = AuthorizerWrapper.convertToResource(action.resourcePattern)
def logMessage: String = { def logMessage: String = {
val authResult = if (authorized) "Allowed" else "Denied" val authResult = if (authorized) "Allowed" else "Denied"
s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource" s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource"
@ -99,14 +99,14 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
override def addAcls(acls: Set[Acl], resource: Resource): Unit = { override def addAcls(acls: Set[Acl], resource: Resource): Unit = {
aclAuthorizer.maxUpdateRetries = maxUpdateRetries aclAuthorizer.maxUpdateRetries = maxUpdateRetries
if (acls != null && acls.nonEmpty) { if (acls != null && acls.nonEmpty) {
val bindings = acls.map { acl => AuthorizerUtils.convertToAclBinding(resource, acl) } val bindings = acls.map { acl => AuthorizerWrapper.convertToAclBinding(resource, acl) }
createAcls(bindings) createAcls(bindings)
} }
} }
override def removeAcls(aclsTobeRemoved: Set[Acl], resource: Resource): Boolean = { override def removeAcls(aclsTobeRemoved: Set[Acl], resource: Resource): Boolean = {
val filters = aclsTobeRemoved.map { acl => val filters = aclsTobeRemoved.map { acl =>
new AclBindingFilter(resource.toPattern.toFilter, AuthorizerUtils.convertToAccessControlEntry(acl).toFilter) new AclBindingFilter(resource.toPattern.toFilter, AuthorizerWrapper.convertToAccessControlEntry(acl).toFilter)
} }
deleteAcls(filters) deleteAcls(filters)
} }
@ -158,8 +158,8 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
private def acls(filter: AclBindingFilter): Map[Resource, Set[Acl]] = { private def acls(filter: AclBindingFilter): Map[Resource, Set[Acl]] = {
val result = mutable.Map[Resource, mutable.Set[Acl]]() val result = mutable.Map[Resource, mutable.Set[Acl]]()
aclAuthorizer.acls(filter).asScala.foreach { binding => aclAuthorizer.acls(filter).asScala.foreach { binding =>
val resource = AuthorizerUtils.convertToResource(binding.pattern) val resource = AuthorizerWrapper.convertToResource(binding.pattern)
val acl = AuthorizerUtils.convertToAcl(binding.entry) val acl = AuthorizerWrapper.convertToAcl(binding.entry)
result.getOrElseUpdate(resource, mutable.Set()).add(acl) result.getOrElseUpdate(resource, mutable.Set()).add(acl)
} }
result.mapValues(_.toSet).toMap result.mapValues(_.toSet).toMap

View File

@ -23,21 +23,22 @@ import java.util.concurrent.locks.ReentrantReadWriteLock
import com.typesafe.scalalogging.Logger import com.typesafe.scalalogging.Logger
import kafka.api.KAFKA_2_0_IV1 import kafka.api.KAFKA_2_0_IV1
import kafka.security.authorizer.AclAuthorizer.VersionedAcls import kafka.security.authorizer.AclAuthorizer.VersionedAcls
import kafka.security.auth.{Acl, Operation, PermissionType, Resource, ResourceType} import kafka.security.authorizer.AclEntry.ResourceSeparator
import kafka.security.auth.{All, Allow, Alter, AlterConfigs, Delete, Deny, Describe, DescribeConfigs, Read, Write}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils._ import kafka.utils._
import kafka.zk._ import kafka.zk._
import org.apache.kafka.common.Endpoint import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl._
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.errors.{ApiException, InvalidRequestException, UnsupportedVersionException} import org.apache.kafka.common.errors.{ApiException, InvalidRequestException, UnsupportedVersionException}
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.resource.{PatternType, ResourcePatternFilter} import org.apache.kafka.common.resource._
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.{Time, SecurityUtils => JSecurityUtils} import org.apache.kafka.common.utils.{Time, SecurityUtils}
import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult
import org.apache.kafka.server.authorizer.{AclCreateResult, AclDeleteResult, Action, AuthorizableRequestContext, AuthorizationResult, Authorizer, AuthorizerServerInfo} import org.apache.kafka.server.authorizer._
import scala.collection.mutable import scala.collection.mutable
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -57,20 +58,21 @@ object AclAuthorizer {
// If set to true when no acls are found for a resource, authorizer allows access to everyone. Defaults to false. // If set to true when no acls are found for a resource, authorizer allows access to everyone. Defaults to false.
val AllowEveryoneIfNoAclIsFoundProp = "allow.everyone.if.no.acl.found" val AllowEveryoneIfNoAclIsFoundProp = "allow.everyone.if.no.acl.found"
case class VersionedAcls(acls: Set[Acl], zkVersion: Int) { case class VersionedAcls(acls: Set[AclEntry], zkVersion: Int) {
def exists: Boolean = zkVersion != ZkVersion.UnknownVersion def exists: Boolean = zkVersion != ZkVersion.UnknownVersion
} }
val NoAcls = VersionedAcls(Set.empty, ZkVersion.UnknownVersion) val NoAcls = VersionedAcls(Set.empty, ZkVersion.UnknownVersion)
val WildcardHost = "*"
// Orders by resource type, then resource pattern type and finally reverse ordering by name. // Orders by resource type, then resource pattern type and finally reverse ordering by name.
private object ResourceOrdering extends Ordering[Resource] { private object ResourceOrdering extends Ordering[ResourcePattern] {
def compare(a: Resource, b: Resource): Int = { def compare(a: ResourcePattern, b: ResourcePattern): Int = {
val rt = a.resourceType compare b.resourceType val rt = a.resourceType.compareTo(b.resourceType)
if (rt != 0) if (rt != 0)
rt rt
else { else {
val rnt = a.patternType compareTo b.patternType val rnt = a.patternType.compareTo(b.patternType)
if (rnt != 0) if (rnt != 0)
rnt rnt
else else
@ -89,7 +91,7 @@ class AclAuthorizer extends Authorizer with Logging {
private var extendedAclSupport: Boolean = _ private var extendedAclSupport: Boolean = _
@volatile @volatile
private var aclCache = new scala.collection.immutable.TreeMap[Resource, VersionedAcls]()(AclAuthorizer.ResourceOrdering) private var aclCache = new scala.collection.immutable.TreeMap[ResourcePattern, VersionedAcls]()(AclAuthorizer.ResourceOrdering)
private val lock = new ReentrantReadWriteLock() private val lock = new ReentrantReadWriteLock()
// The maximum number of times we should try to update the resource acls in zookeeper before failing; // The maximum number of times we should try to update the resource acls in zookeeper before failing;
@ -108,7 +110,7 @@ class AclAuthorizer extends Authorizer with Logging {
configs.foreach { case (key, value) => props.put(key, value.toString) } configs.foreach { case (key, value) => props.put(key, value.toString) }
superUsers = configs.get(AclAuthorizer.SuperUsersProp).collect { superUsers = configs.get(AclAuthorizer.SuperUsersProp).collect {
case str: String if str.nonEmpty => str.split(";").map(s => JSecurityUtils.parseKafkaPrincipal(s.trim)).toSet case str: String if str.nonEmpty => str.split(";").map(s => SecurityUtils.parseKafkaPrincipal(s.trim)).toSet
}.getOrElse(Set.empty[KafkaPrincipal]) }.getOrElse(Set.empty[KafkaPrincipal])
shouldAllowEveryoneIfNoAclIsFound = configs.get(AclAuthorizer.AllowEveryoneIfNoAclIsFoundProp).exists(_.toString.toBoolean) shouldAllowEveryoneIfNoAclIsFound = configs.get(AclAuthorizer.AllowEveryoneIfNoAclIsFoundProp).exists(_.toString.toBoolean)
@ -167,8 +169,8 @@ class AclAuthorizer extends Authorizer with Logging {
inWriteLock(lock) { inWriteLock(lock) {
aclsToCreate.foreach { case (resource, aclsWithIndex) => aclsToCreate.foreach { case (resource, aclsWithIndex) =>
try { try {
updateResourceAcls(AuthorizerUtils.convertToResource(resource)) { currentAcls => updateResourceAcls(resource) { currentAcls =>
val newAcls = aclsWithIndex.map { case (acl, index) => AuthorizerUtils.convertToAcl(acl.entry) } val newAcls = aclsWithIndex.map { case (acl, index) => new AclEntry(acl.entry) }
currentAcls ++ newAcls currentAcls ++ newAcls
} }
aclsWithIndex.foreach { case (_, index) => results(index) = AclCreateResult.SUCCESS } aclsWithIndex.foreach { case (_, index) => results(index) = AclCreateResult.SUCCESS }
@ -192,7 +194,7 @@ class AclAuthorizer extends Authorizer with Logging {
val resources = aclCache.keys ++ filters.map(_._1.patternFilter).filter(_.matchesAtMostOne).flatMap(filterToResources) val resources = aclCache.keys ++ filters.map(_._1.patternFilter).filter(_.matchesAtMostOne).flatMap(filterToResources)
val resourcesToUpdate = resources.map { resource => val resourcesToUpdate = resources.map { resource =>
val matchingFilters = filters.filter { case (filter, _) => val matchingFilters = filters.filter { case (filter, _) =>
filter.patternFilter.matches(resource.toPattern) filter.patternFilter.matches(resource)
} }
resource -> matchingFilters resource -> matchingFilters
}.toMap.filter(_._2.nonEmpty) }.toMap.filter(_._2.nonEmpty)
@ -203,9 +205,9 @@ class AclAuthorizer extends Authorizer with Logging {
updateResourceAcls(resource) { currentAcls => updateResourceAcls(resource) { currentAcls =>
val aclsToRemove = currentAcls.filter { acl => val aclsToRemove = currentAcls.filter { acl =>
matchingFilters.exists { case (filter, index) => matchingFilters.exists { case (filter, index) =>
val matches = filter.entryFilter.matches(AuthorizerUtils.convertToAccessControlEntry(acl)) val matches = filter.entryFilter.matches(acl)
if (matches) { if (matches) {
val binding = AuthorizerUtils.convertToAclBinding(resource, acl) val binding = new AclBinding(resource, acl)
deletedBindings.getOrElseUpdate(binding, index) deletedBindings.getOrElseUpdate(binding, index)
resourceBindingsBeingDeleted.getOrElseUpdate(binding, index) resourceBindingsBeingDeleted.getOrElseUpdate(binding, index)
} }
@ -232,7 +234,7 @@ class AclAuthorizer extends Authorizer with Logging {
override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = { override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = {
inReadLock(lock) { inReadLock(lock) {
unorderedAcls.flatMap { case (resource, versionedAcls) => unorderedAcls.flatMap { case (resource, versionedAcls) =>
versionedAcls.acls.map(acl => AuthorizerUtils.convertToAclBinding(resource, acl)) versionedAcls.acls.map(acl => new AclBinding(resource, acl.ace))
.filter(filter.matches) .filter(filter.matches)
}.asJava }.asJava
} }
@ -244,7 +246,7 @@ class AclAuthorizer extends Authorizer with Logging {
} }
private def authorizeAction(requestContext: AuthorizableRequestContext, action: Action): AuthorizationResult = { private def authorizeAction(requestContext: AuthorizableRequestContext, action: Action): AuthorizationResult = {
val resource = AuthorizerUtils.convertToResource(action.resourcePattern) val resource = action.resourcePattern
if (resource.patternType != PatternType.LITERAL) { if (resource.patternType != PatternType.LITERAL) {
throw new IllegalArgumentException("Only literal resources are supported. Got: " + resource.patternType) throw new IllegalArgumentException("Only literal resources are supported. Got: " + resource.patternType)
} }
@ -257,9 +259,9 @@ class AclAuthorizer extends Authorizer with Logging {
sessionPrincipal sessionPrincipal
val host = requestContext.clientAddress.getHostAddress val host = requestContext.clientAddress.getHostAddress
val operation = Operation.fromJava(action.operation) val operation = action.operation
def isEmptyAclAndAuthorized(acls: Set[Acl]): Boolean = { def isEmptyAclAndAuthorized(acls: Set[AclEntry]): Boolean = {
if (acls.isEmpty) { if (acls.isEmpty) {
// No ACLs found for this resource, permission is determined by value of config allow.everyone.if.no.acl.found // No ACLs found for this resource, permission is determined by value of config allow.everyone.if.no.acl.found
authorizerLogger.debug(s"No acl found for resource $resource, authorized = $shouldAllowEveryoneIfNoAclIsFound") authorizerLogger.debug(s"No acl found for resource $resource, authorized = $shouldAllowEveryoneIfNoAclIsFound")
@ -267,21 +269,21 @@ class AclAuthorizer extends Authorizer with Logging {
} else false } else false
} }
def denyAclExists(acls: Set[Acl]): Boolean = { def denyAclExists(acls: Set[AclEntry]): Boolean = {
// Check if there are any Deny ACLs which would forbid this operation. // Check if there are any Deny ACLs which would forbid this operation.
matchingAclExists(operation, resource, principal, host, Deny, acls) matchingAclExists(operation, resource, principal, host, DENY, acls)
} }
def allowAclExists(acls: Set[Acl]): Boolean = { def allowAclExists(acls: Set[AclEntry]): Boolean = {
// Check if there are any Allow ACLs which would allow this operation. // Check if there are any Allow ACLs which would allow this operation.
// Allowing read, write, delete, or alter implies allowing describe. // Allowing read, write, delete, or alter implies allowing describe.
// See #{org.apache.kafka.common.acl.AclOperation} for more details about ACL inheritance. // See #{org.apache.kafka.common.acl.AclOperation} for more details about ACL inheritance.
val allowOps = operation match { val allowOps = operation match {
case Describe => Set[Operation](Describe, Read, Write, Delete, Alter) case DESCRIBE => Set[AclOperation](DESCRIBE, READ, WRITE, DELETE, ALTER)
case DescribeConfigs => Set[Operation](DescribeConfigs, AlterConfigs) case DESCRIBE_CONFIGS => Set[AclOperation](DESCRIBE_CONFIGS, ALTER_CONFIGS)
case _ => Set[Operation](operation) case _ => Set[AclOperation](operation)
} }
allowOps.exists(operation => matchingAclExists(operation, resource, principal, host, Allow, acls)) allowOps.exists(operation => matchingAclExists(operation, resource, principal, host, ALLOW, acls))
} }
def aclsAllowAccess = { def aclsAllowAccess = {
@ -305,19 +307,19 @@ class AclAuthorizer extends Authorizer with Logging {
} else false } else false
} }
private def matchingAcls(resourceType: ResourceType, resourceName: String): Set[Acl] = { private def matchingAcls(resourceType: ResourceType, resourceName: String): Set[AclEntry] = {
inReadLock(lock) { inReadLock(lock) {
val wildcard = aclCache.get(Resource(resourceType, Acl.WildCardResource, PatternType.LITERAL)) val wildcard = aclCache.get(new ResourcePattern(resourceType, ResourcePattern.WILDCARD_RESOURCE, PatternType.LITERAL))
.map(_.acls) .map(_.acls)
.getOrElse(Set.empty[Acl]) .getOrElse(Set.empty)
val literal = aclCache.get(Resource(resourceType, resourceName, PatternType.LITERAL)) val literal = aclCache.get(new ResourcePattern(resourceType, resourceName, PatternType.LITERAL))
.map(_.acls) .map(_.acls)
.getOrElse(Set.empty[Acl]) .getOrElse(Set.empty)
val prefixed = aclCache val prefixed = aclCache
.from(Resource(resourceType, resourceName, PatternType.PREFIXED)) .from(new ResourcePattern(resourceType, resourceName, PatternType.PREFIXED))
.to(Resource(resourceType, resourceName.take(1), PatternType.PREFIXED)) .to(new ResourcePattern(resourceType, resourceName.take(1), PatternType.PREFIXED))
.filterKeys(resource => resourceName.startsWith(resource.name)) .filterKeys(resource => resourceName.startsWith(resource.name))
.values .values
.flatMap { _.acls } .flatMap { _.acls }
@ -327,12 +329,17 @@ class AclAuthorizer extends Authorizer with Logging {
} }
} }
private def matchingAclExists(operation: Operation, resource: Resource, principal: KafkaPrincipal, host: String, permissionType: PermissionType, acls: Set[Acl]): Boolean = { private def matchingAclExists(operation: AclOperation,
resource: ResourcePattern,
principal: KafkaPrincipal,
host: String,
permissionType: AclPermissionType,
acls: Set[AclEntry]): Boolean = {
acls.find { acl => acls.find { acl =>
acl.permissionType == permissionType && acl.permissionType == permissionType &&
(acl.principal == principal || acl.principal == Acl.WildCardPrincipal) && (acl.kafkaPrincipal == principal || acl.kafkaPrincipal == AclEntry.WildcardPrincipal) &&
(operation == acl.operation || acl.operation == All) && (operation == acl.operation || acl.operation == AclOperation.ALL) &&
(acl.host == host || acl.host == Acl.WildCardHost) (acl.host == host || acl.host == AclEntry.WildcardHost)
}.exists { acl => }.exists { acl =>
authorizerLogger.debug(s"operation = $operation on resource = $resource from host = $host is $permissionType based on acl = $acl") authorizerLogger.debug(s"operation = $operation on resource = $resource from host = $host is $permissionType based on acl = $acl")
true true
@ -344,12 +351,12 @@ class AclAuthorizer extends Authorizer with Logging {
ZkAclStore.stores.foreach(store => { ZkAclStore.stores.foreach(store => {
val resourceTypes = zkClient.getResourceTypes(store.patternType) val resourceTypes = zkClient.getResourceTypes(store.patternType)
for (rType <- resourceTypes) { for (rType <- resourceTypes) {
val resourceType = Try(ResourceType.fromString(rType)) val resourceType = Try(SecurityUtils.resourceType(rType))
resourceType match { resourceType match {
case Success(resourceTypeObj) => case Success(resourceTypeObj) =>
val resourceNames = zkClient.getResourceNames(store.patternType, resourceTypeObj) val resourceNames = zkClient.getResourceNames(store.patternType, resourceTypeObj)
for (resourceName <- resourceNames) { for (resourceName <- resourceNames) {
val resource = new Resource(resourceTypeObj, resourceName, store.patternType) val resource = new ResourcePattern(resourceTypeObj, resourceName, store.patternType)
val versionedAcls = getAclsFromZk(resource) val versionedAcls = getAclsFromZk(resource)
updateCache(resource, versionedAcls) updateCache(resource, versionedAcls)
} }
@ -365,13 +372,13 @@ class AclAuthorizer extends Authorizer with Logging {
.map(store => store.createListener(AclChangedNotificationHandler, zkClient)) .map(store => store.createListener(AclChangedNotificationHandler, zkClient))
} }
private def filterToResources(filter: ResourcePatternFilter): Set[Resource] = { private def filterToResources(filter: ResourcePatternFilter): Set[ResourcePattern] = {
filter.patternType match { filter.patternType match {
case PatternType.LITERAL | PatternType.PREFIXED => case PatternType.LITERAL | PatternType.PREFIXED =>
Set(Resource(ResourceType.fromJava(filter.resourceType), filter.name, filter.patternType)) Set(new ResourcePattern(filter.resourceType, filter.name, filter.patternType))
case PatternType.ANY => case PatternType.ANY =>
Set(Resource(ResourceType.fromJava(filter.resourceType), filter.name, PatternType.LITERAL), Set(new ResourcePattern(filter.resourceType, filter.name, PatternType.LITERAL),
Resource(ResourceType.fromJava(filter.resourceType), filter.name, PatternType.PREFIXED)) new ResourcePattern(filter.resourceType, filter.name, PatternType.PREFIXED))
case _ => throw new IllegalArgumentException(s"Cannot determine matching resources for patternType $filter") case _ => throw new IllegalArgumentException(s"Cannot determine matching resources for patternType $filter")
} }
} }
@ -379,12 +386,14 @@ class AclAuthorizer extends Authorizer with Logging {
def logAuditMessage(requestContext: AuthorizableRequestContext, action: Action, authorized: Boolean): Unit = { def logAuditMessage(requestContext: AuthorizableRequestContext, action: Action, authorized: Boolean): Unit = {
def logMessage: String = { def logMessage: String = {
val principal = requestContext.principal val principal = requestContext.principal
val operation = Operation.fromJava(action.operation) val operation = SecurityUtils.operationName(action.operation)
val host = requestContext.clientAddress.getHostAddress val host = requestContext.clientAddress.getHostAddress
val resource = AuthorizerUtils.convertToResource(action.resourcePattern) val resourceType = SecurityUtils.resourceTypeName(action.resourcePattern.resourceType)
val resource = s"$resourceType$ResourceSeparator${action.resourcePattern.patternType}$ResourceSeparator${action.resourcePattern.name}"
val authResult = if (authorized) "Allowed" else "Denied" val authResult = if (authorized) "Allowed" else "Denied"
val apiKey = if (ApiKeys.hasId(requestContext.requestType)) ApiKeys.forId(requestContext.requestType).name else requestContext.requestType val apiKey = if (ApiKeys.hasId(requestContext.requestType)) ApiKeys.forId(requestContext.requestType).name else requestContext.requestType
val refCount = action.resourceReferenceCount val refCount = action.resourceReferenceCount
s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource for request = $apiKey with resourceRefCount = $refCount" s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource for request = $apiKey with resourceRefCount = $refCount"
} }
@ -418,7 +427,7 @@ class AclAuthorizer extends Authorizer with Logging {
* @param getNewAcls function to transform existing acls to new ACLs * @param getNewAcls function to transform existing acls to new ACLs
* @return boolean indicating if a change was made * @return boolean indicating if a change was made
*/ */
private def updateResourceAcls(resource: Resource)(getNewAcls: Set[Acl] => Set[Acl]): Boolean = { private def updateResourceAcls(resource: ResourcePattern)(getNewAcls: Set[AclEntry] => Set[AclEntry]): Boolean = {
var currentVersionedAcls = var currentVersionedAcls =
if (aclCache.contains(resource)) if (aclCache.contains(resource))
getAclsFromCache(resource) getAclsFromCache(resource)
@ -468,17 +477,17 @@ class AclAuthorizer extends Authorizer with Logging {
// Returns Map instead of SortedMap since most callers don't care about ordering. In Scala 2.13, mapping from SortedMap // Returns Map instead of SortedMap since most callers don't care about ordering. In Scala 2.13, mapping from SortedMap
// to Map is restricted by default // to Map is restricted by default
private def unorderedAcls: Map[Resource, VersionedAcls] = aclCache private def unorderedAcls: Map[ResourcePattern, VersionedAcls] = aclCache
private def getAclsFromCache(resource: Resource): VersionedAcls = { private def getAclsFromCache(resource: ResourcePattern): VersionedAcls = {
aclCache.getOrElse(resource, throw new IllegalArgumentException(s"ACLs do not exist in the cache for resource $resource")) aclCache.getOrElse(resource, throw new IllegalArgumentException(s"ACLs do not exist in the cache for resource $resource"))
} }
private def getAclsFromZk(resource: Resource): VersionedAcls = { private def getAclsFromZk(resource: ResourcePattern): VersionedAcls = {
zkClient.getVersionedAclsForResource(resource) zkClient.getVersionedAclsForResource(resource)
} }
private def updateCache(resource: Resource, versionedAcls: VersionedAcls): Unit = { private def updateCache(resource: ResourcePattern, versionedAcls: VersionedAcls): Unit = {
if (versionedAcls.acls.nonEmpty) { if (versionedAcls.acls.nonEmpty) {
aclCache = aclCache + (resource -> versionedAcls) aclCache = aclCache + (resource -> versionedAcls)
} else { } else {
@ -486,7 +495,7 @@ class AclAuthorizer extends Authorizer with Logging {
} }
} }
private def updateAclChangedFlag(resource: Resource): Unit = { private def updateAclChangedFlag(resource: ResourcePattern): Unit = {
zkClient.createAclChangeNotification(resource) zkClient.createAclChangeNotification(resource)
} }
@ -502,7 +511,7 @@ class AclAuthorizer extends Authorizer with Logging {
} }
object AclChangedNotificationHandler extends AclChangeNotificationHandler { object AclChangedNotificationHandler extends AclChangeNotificationHandler {
override def processNotification(resource: Resource): Unit = { override def processNotification(resource: ResourcePattern): Unit = {
inWriteLock(lock) { inWriteLock(lock) {
val versionedAcls = getAclsFromZk(resource) val versionedAcls = getAclsFromZk(resource)
updateCache(resource, versionedAcls) updateCache(resource, versionedAcls)

View File

@ -0,0 +1,146 @@
/**
* 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 kafka.security.authorizer
import kafka.utils.Json
import org.apache.kafka.common.acl.{AccessControlEntry, AclOperation, AclPermissionType}
import org.apache.kafka.common.acl.AclOperation.{READ, WRITE, CREATE, DESCRIBE, DELETE, ALTER, DESCRIBE_CONFIGS, ALTER_CONFIGS, CLUSTER_ACTION, IDEMPOTENT_WRITE}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.resource.{ResourcePattern, ResourceType}
import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.SecurityUtils
import scala.collection.JavaConverters._
object AclEntry {
val WildcardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*")
val WildcardPrincipalString: String = WildcardPrincipal.toString
val WildcardHost: String = "*"
val WildcardResource: String = ResourcePattern.WILDCARD_RESOURCE
val ResourceSeparator = ":"
val ResourceTypes: Set[ResourceType] = ResourceType.values.toSet
.filterNot(t => t == ResourceType.UNKNOWN || t == ResourceType.ANY)
val AclOperations: Set[AclOperation] = AclOperation.values.toSet
.filterNot(t => t == AclOperation.UNKNOWN || t == AclOperation.ANY)
val PrincipalKey = "principal"
val PermissionTypeKey = "permissionType"
val OperationKey = "operation"
val HostsKey = "host"
val VersionKey = "version"
val CurrentVersion = 1
val AclsKey = "acls"
def apply(principal: KafkaPrincipal,
permissionType: AclPermissionType,
host: String,
operation: AclOperation): AclEntry = {
new AclEntry(new AccessControlEntry(if (principal == null) null else principal.toString,
host, operation, permissionType))
}
/**
* Parse JSON representation of ACLs
* @param bytes of acls json string
*
* <p>
{
"version": 1,
"acls": [
{
"host":"host1",
"permissionType": "Deny",
"operation": "Read",
"principal": "User:alice"
}
]
}
* </p>
*
* @return set of AclEntry objects from the JSON string
*/
def fromBytes(bytes: Array[Byte]): Set[AclEntry] = {
if (bytes == null || bytes.isEmpty)
return collection.immutable.Set.empty[AclEntry]
Json.parseBytes(bytes).map(_.asJsonObject).map { js =>
//the acl json version.
require(js(VersionKey).to[Int] == CurrentVersion)
js(AclsKey).asJsonArray.iterator.map(_.asJsonObject).map { itemJs =>
val principal = SecurityUtils.parseKafkaPrincipal(itemJs(PrincipalKey).to[String])
val permissionType = SecurityUtils.permissionType(itemJs(PermissionTypeKey).to[String])
val host = itemJs(HostsKey).to[String]
val operation = SecurityUtils.operation(itemJs(OperationKey).to[String])
AclEntry(principal, permissionType, host, operation)
}.toSet
}.getOrElse(Set.empty)
}
def toJsonCompatibleMap(acls: Set[AclEntry]): Map[String, Any] = {
Map(AclEntry.VersionKey -> AclEntry.CurrentVersion, AclEntry.AclsKey -> acls.map(acl => acl.toMap.asJava).toList.asJava)
}
def supportedOperations(resourceType: ResourceType): Set[AclOperation] = {
resourceType match {
case ResourceType.TOPIC => Set(READ, WRITE, CREATE, DESCRIBE, DELETE, ALTER, DESCRIBE_CONFIGS, ALTER_CONFIGS)
case ResourceType.GROUP => Set(READ, DESCRIBE, DELETE)
case ResourceType.CLUSTER => Set(CREATE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, ALTER, DESCRIBE)
case ResourceType.TRANSACTIONAL_ID => Set(DESCRIBE, WRITE)
case ResourceType.DELEGATION_TOKEN => Set(DESCRIBE)
case _ => throw new IllegalArgumentException("Not a concrete resource type")
}
}
def authorizationError(resourceType: ResourceType): Errors = {
resourceType match {
case ResourceType.TOPIC => Errors.TOPIC_AUTHORIZATION_FAILED
case ResourceType.GROUP => Errors.GROUP_AUTHORIZATION_FAILED
case ResourceType.CLUSTER => Errors.CLUSTER_AUTHORIZATION_FAILED
case ResourceType.TRANSACTIONAL_ID => Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED
case ResourceType.DELEGATION_TOKEN => Errors.DELEGATION_TOKEN_AUTHORIZATION_FAILED
case _ => throw new IllegalArgumentException("Authorization error type not known")
}
}
}
class AclEntry(val ace: AccessControlEntry)
extends AccessControlEntry(ace.principal, ace.host, ace.operation, ace.permissionType) {
val kafkaPrincipal: KafkaPrincipal = if (principal == null)
null
else
SecurityUtils.parseKafkaPrincipal(principal)
def toMap: Map[String, Any] = {
Map(AclEntry.PrincipalKey -> principal,
AclEntry.PermissionTypeKey -> SecurityUtils.permissionTypeName(permissionType),
AclEntry.OperationKey -> SecurityUtils.operationName(operation),
AclEntry.HostsKey -> host)
}
override def hashCode(): Int = ace.hashCode()
override def equals(o: scala.Any): Boolean = super.equals(o) // to keep spotbugs happy
override def toString: String = {
"%s has %s permission for operations: %s from hosts: %s".format(principal, permissionType.name, operation, host)
}
}

View File

@ -20,65 +20,31 @@ package kafka.security.authorizer
import java.net.InetAddress import java.net.InetAddress
import kafka.network.RequestChannel.Session import kafka.network.RequestChannel.Session
import kafka.security.auth._ import kafka.security.auth.{Authorizer => LegacyAuthorizer}
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation} import org.apache.kafka.common.acl._
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.requests.ApiError import org.apache.kafka.common.resource.Resource
import org.apache.kafka.common.resource.{ResourcePattern, ResourceType => JResourceType}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.SecurityUtils._ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.authorizer.AuthorizableRequestContext import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer}
import scala.util.{Failure, Success, Try}
object AuthorizerUtils { object AuthorizerUtils {
val WildcardPrincipal = "User:*"
val WildcardHost = "*"
def convertToResourceAndAcl(filter: AclBindingFilter): Either[ApiError, (Resource, Acl)] = { def createAuthorizer(className: String): Authorizer = {
(for { Utils.newInstance(className, classOf[Object]) match {
resourceType <- Try(ResourceType.fromJava(filter.patternFilter.resourceType)) case auth: Authorizer => auth
principal <- Try(parseKafkaPrincipal(filter.entryFilter.principal)) case auth: kafka.security.auth.Authorizer => new AuthorizerWrapper(auth)
operation <- Try(Operation.fromJava(filter.entryFilter.operation)) case _ => throw new ConfigException(s"Authorizer does not implement ${classOf[Authorizer].getName} or ${classOf[LegacyAuthorizer].getName}.")
permissionType <- Try(PermissionType.fromJava(filter.entryFilter.permissionType))
resource = Resource(resourceType, filter.patternFilter.name, filter.patternFilter.patternType)
acl = Acl(principal, permissionType, filter.entryFilter.host, operation)
} yield (resource, acl)) match {
case Failure(throwable) => Left(new ApiError(Errors.INVALID_REQUEST, throwable.getMessage))
case Success(s) => Right(s)
} }
} }
def convertToAclBinding(resource: Resource, acl: Acl): AclBinding = {
val resourcePattern = new ResourcePattern(resource.resourceType.toJava, resource.name, resource.patternType)
new AclBinding(resourcePattern, convertToAccessControlEntry(acl))
}
def convertToAccessControlEntry(acl: Acl): AccessControlEntry = {
new AccessControlEntry(acl.principal.toString, acl.host.toString,
acl.operation.toJava, acl.permissionType.toJava)
}
def convertToAcl(ace: AccessControlEntry): Acl = {
new Acl(parseKafkaPrincipal(ace.principal), PermissionType.fromJava(ace.permissionType), ace.host,
Operation.fromJava(ace.operation))
}
def convertToResource(resourcePattern: ResourcePattern): Resource = {
Resource(ResourceType.fromJava(resourcePattern.resourceType), resourcePattern.name, resourcePattern.patternType)
}
def validateAclBinding(aclBinding: AclBinding): Unit = { def validateAclBinding(aclBinding: AclBinding): Unit = {
if (aclBinding.isUnknown) if (aclBinding.isUnknown)
throw new IllegalArgumentException("ACL binding contains unknown elements") throw new IllegalArgumentException("ACL binding contains unknown elements")
} }
def supportedOperations(resourceType: JResourceType): Set[AclOperation] = { def isClusterResource(name: String): Boolean = name.equals(Resource.CLUSTER_NAME)
ResourceType.fromJava(resourceType).supportedOperations.map(_.toJava)
}
def isClusterResource(name: String): Boolean = name.equals(Resource.ClusterResourceName)
def sessionToRequestContext(session: Session): AuthorizableRequestContext = { def sessionToRequestContext(session: Session): AuthorizableRequestContext = {
new AuthorizableRequestContext { new AuthorizableRequestContext {

View File

@ -21,19 +21,61 @@ import java.util.concurrent.{CompletableFuture, CompletionStage}
import java.{lang, util} import java.{lang, util}
import kafka.network.RequestChannel.Session import kafka.network.RequestChannel.Session
import kafka.security.auth.{Acl, Operation, Resource} import kafka.security.auth.{Acl, Operation, PermissionType, Resource, ResourceType}
import kafka.security.authorizer.AuthorizerWrapper._
import org.apache.kafka.common.Endpoint import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter}
import org.apache.kafka.common.errors.{ApiException, InvalidRequestException} import org.apache.kafka.common.errors.{ApiException, InvalidRequestException}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.ApiError import org.apache.kafka.common.requests.ApiError
import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.utils.SecurityUtils.parseKafkaPrincipal
import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, AuthorizerServerInfo, _} import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, AuthorizerServerInfo, _}
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.collection.{Seq, immutable, mutable} import scala.collection.{Seq, immutable, mutable}
import scala.util.{Failure, Success, Try}
@deprecated("Use kafka.security.authorizer.AclAuthorizer", "Since 2.5")
object AuthorizerWrapper {
def convertToResourceAndAcl(filter: AclBindingFilter): Either[ApiError, (Resource, Acl)] = {
(for {
resourceType <- Try(ResourceType.fromJava(filter.patternFilter.resourceType))
principal <- Try(parseKafkaPrincipal(filter.entryFilter.principal))
operation <- Try(Operation.fromJava(filter.entryFilter.operation))
permissionType <- Try(PermissionType.fromJava(filter.entryFilter.permissionType))
resource = Resource(resourceType, filter.patternFilter.name, filter.patternFilter.patternType)
acl = Acl(principal, permissionType, filter.entryFilter.host, operation)
} yield (resource, acl)) match {
case Failure(throwable) => Left(new ApiError(Errors.INVALID_REQUEST, throwable.getMessage))
case Success(s) => Right(s)
}
}
def convertToAclBinding(resource: Resource, acl: Acl): AclBinding = {
val resourcePattern = new ResourcePattern(resource.resourceType.toJava, resource.name, resource.patternType)
new AclBinding(resourcePattern, convertToAccessControlEntry(acl))
}
def convertToAccessControlEntry(acl: Acl): AccessControlEntry = {
new AccessControlEntry(acl.principal.toString, acl.host.toString,
acl.operation.toJava, acl.permissionType.toJava)
}
def convertToAcl(ace: AccessControlEntry): Acl = {
new Acl(parseKafkaPrincipal(ace.principal), PermissionType.fromJava(ace.permissionType), ace.host,
Operation.fromJava(ace.operation))
}
def convertToResource(resourcePattern: ResourcePattern): Resource = {
Resource(ResourceType.fromJava(resourcePattern.resourceType), resourcePattern.name, resourcePattern.patternType)
}
}
@deprecated("Use kafka.security.authorizer.AclAuthorizer", "Since 2.5")
class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.Authorizer) extends Authorizer { class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.Authorizer) extends Authorizer {
override def configure(configs: util.Map[String, _]): Unit = { override def configure(configs: util.Map[String, _]): Unit = {
@ -49,7 +91,7 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A
val session = Session(requestContext.principal, requestContext.clientAddress) val session = Session(requestContext.principal, requestContext.clientAddress)
actions.asScala.map { action => actions.asScala.map { action =>
val operation = Operation.fromJava(action.operation) val operation = Operation.fromJava(action.operation)
if (baseAuthorizer.authorize(session, operation, AuthorizerUtils.convertToResource(action.resourcePattern))) if (baseAuthorizer.authorize(session, operation, convertToResource(action.resourcePattern)))
AuthorizationResult.ALLOWED AuthorizationResult.ALLOWED
else else
AuthorizationResult.DENIED AuthorizationResult.DENIED
@ -60,7 +102,7 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A
aclBindings: util.List[AclBinding]): util.List[_ <: CompletionStage[AclCreateResult]] = { aclBindings: util.List[AclBinding]): util.List[_ <: CompletionStage[AclCreateResult]] = {
aclBindings.asScala aclBindings.asScala
.map { aclBinding => .map { aclBinding =>
AuthorizerUtils.convertToResourceAndAcl(aclBinding.toFilter) match { convertToResourceAndAcl(aclBinding.toFilter) match {
case Left(apiError) => new AclCreateResult(apiError.exception) case Left(apiError) => new AclCreateResult(apiError.exception)
case Right((resource, acl)) => case Right((resource, acl)) =>
try { try {
@ -83,7 +125,7 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A
if (filters.forall(_.matchesAtMostOne)) { if (filters.forall(_.matchesAtMostOne)) {
// Delete based on a list of ACL fixtures. // Delete based on a list of ACL fixtures.
for ((filter, i) <- filters.zipWithIndex) { for ((filter, i) <- filters.zipWithIndex) {
AuthorizerUtils.convertToResourceAndAcl(filter) match { convertToResourceAndAcl(filter) match {
case Left(apiError) => results.put(i, new AclDeleteResult(apiError.exception)) case Left(apiError) => results.put(i, new AclDeleteResult(apiError.exception))
case Right(binding) => toDelete.put(i, ArrayBuffer(binding)) case Right(binding) => toDelete.put(i, ArrayBuffer(binding))
} }
@ -105,7 +147,7 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A
for ((i, acls) <- toDelete) { for ((i, acls) <- toDelete) {
val deletionResults = acls.flatMap { case (resource, acl) => val deletionResults = acls.flatMap { case (resource, acl) =>
val aclBinding = AuthorizerUtils.convertToAclBinding(resource, acl) val aclBinding = convertToAclBinding(resource, acl)
try { try {
if (baseAuthorizer.removeAcls(immutable.Set(acl), resource)) if (baseAuthorizer.removeAcls(immutable.Set(acl), resource))
Some(new AclBindingDeleteResult(aclBinding)) Some(new AclBindingDeleteResult(aclBinding))
@ -126,7 +168,7 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A
override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = { override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = {
baseAuthorizer.getAcls().flatMap { case (resource, acls) => baseAuthorizer.getAcls().flatMap { case (resource, acls) =>
acls.map(acl => AuthorizerUtils.convertToAclBinding(resource, acl)).filter(filter.matches) acls.map(acl => convertToAclBinding(resource, acl)).filter(filter.matches)
}.asJava }.asJava
} }

View File

@ -36,7 +36,7 @@ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinat
import kafka.log.AppendOrigin import kafka.log.AppendOrigin
import kafka.message.ZStdCompressionCodec import kafka.message.ZStdCompressionCodec
import kafka.network.RequestChannel import kafka.network.RequestChannel
import kafka.security.authorizer.AuthorizerUtils import kafka.security.authorizer.{AclEntry, AuthorizerUtils}
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import kafka.zk.{AdminZkClient, KafkaZkClient} import kafka.zk.{AdminZkClient, KafkaZkClient}
@ -2824,7 +2824,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
private def authorizedOperations(request: RequestChannel.Request, resource: Resource): Int = { private def authorizedOperations(request: RequestChannel.Request, resource: Resource): Int = {
val supportedOps = AuthorizerUtils.supportedOperations(resource.resourceType).toList val supportedOps = AclEntry.supportedOperations(resource.resourceType).toList
val authorizedOps = authorizer match { val authorizedOps = authorizer match {
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)

View File

@ -25,7 +25,7 @@ import kafka.cluster.EndPoint
import kafka.coordinator.group.OffsetConfig import kafka.coordinator.group.OffsetConfig
import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager} import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, ZStdCompressionCodec} import kafka.message.{BrokerCompressionCodec, CompressionCodec, ZStdCompressionCodec}
import kafka.security.authorizer.AuthorizerWrapper import kafka.security.authorizer.AuthorizerUtils
import kafka.utils.CoreUtils import kafka.utils.CoreUtils
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.CommonClientConfigs
@ -1183,12 +1183,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
if (className == null || className.isEmpty) if (className == null || className.isEmpty)
None None
else { else {
val authZ = Utils.newInstance(className, classOf[Object]) match { Some(AuthorizerUtils.createAuthorizer(className))
case auth: Authorizer => auth
case auth: kafka.security.auth.Authorizer => new AuthorizerWrapper(auth)
case auth => throw new ConfigException(s"Authorizer does not implement ${classOf[Authorizer].getName} or kafka.security.auth.Authorizer .")
}
Some(authZ)
} }
} }

View File

@ -25,12 +25,12 @@ import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAs
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls} import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls}
import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.security.authorizer.AclEntry
import kafka.server.ConfigType import kafka.server.ConfigType
import kafka.utils.Logging import kafka.utils.Logging
import kafka.zookeeper._ import kafka.zookeeper._
import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.{KafkaException, TopicPartition}
@ -38,6 +38,7 @@ import org.apache.zookeeper.KeeperException.{Code, NodeExistsException}
import org.apache.zookeeper.OpResult.{CreateResult, ErrorResult, SetDataResult} import org.apache.zookeeper.OpResult.{CreateResult, ErrorResult, SetDataResult}
import org.apache.zookeeper.data.{ACL, Stat} import org.apache.zookeeper.data.{ACL, Stat}
import org.apache.zookeeper.{CreateMode, KeeperException, ZooKeeper} import org.apache.zookeeper.{CreateMode, KeeperException, ZooKeeper}
import scala.collection.{Map, Seq, mutable} import scala.collection.{Map, Seq, mutable}
/** /**
@ -1104,7 +1105,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
def createAclPaths(): Unit = { def createAclPaths(): Unit = {
ZkAclStore.stores.foreach(store => { ZkAclStore.stores.foreach(store => {
createRecursive(store.aclPath, throwIfPathExists = false) createRecursive(store.aclPath, throwIfPathExists = false)
ResourceType.values.foreach(resourceType => createRecursive(store.path(resourceType), throwIfPathExists = false)) AclEntry.ResourceTypes.foreach(resourceType => createRecursive(store.path(resourceType), throwIfPathExists = false))
}) })
ZkAclChangeStore.stores.foreach(store => createRecursive(store.aclChangePath, throwIfPathExists = false)) ZkAclChangeStore.stores.foreach(store => createRecursive(store.aclChangePath, throwIfPathExists = false))
@ -1115,7 +1116,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param resource Resource to get VersionedAcls for * @param resource Resource to get VersionedAcls for
* @return VersionedAcls * @return VersionedAcls
*/ */
def getVersionedAclsForResource(resource: Resource): VersionedAcls = { def getVersionedAclsForResource(resource: ResourcePattern): VersionedAcls = {
val getDataRequest = GetDataRequest(ResourceZNode.path(resource)) val getDataRequest = GetDataRequest(ResourceZNode.path(resource))
val getDataResponse = retryRequestUntilConnected(getDataRequest) val getDataResponse = retryRequestUntilConnected(getDataRequest)
getDataResponse.resultCode match { getDataResponse.resultCode match {
@ -1133,7 +1134,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param expectedVersion * @param expectedVersion
* @return true if the update was successful and the new version * @return true if the update was successful and the new version
*/ */
def conditionalSetAclsForResource(resource: Resource, aclsSet: Set[Acl], expectedVersion: Int): (Boolean, Int) = { def conditionalSetAclsForResource(resource: ResourcePattern,
aclsSet: Set[AclEntry],
expectedVersion: Int): (Boolean, Int) = {
def set(aclData: Array[Byte], expectedVersion: Int): SetDataResponse = { def set(aclData: Array[Byte], expectedVersion: Int): SetDataResponse = {
val setDataRequest = SetDataRequest(ResourceZNode.path(resource), aclData, expectedVersion) val setDataRequest = SetDataRequest(ResourceZNode.path(resource), aclData, expectedVersion)
retryRequestUntilConnected(setDataRequest) retryRequestUntilConnected(setDataRequest)
@ -1152,7 +1155,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
} }
} }
def createAclsForResourceIfNotExists(resource: Resource, aclsSet: Set[Acl]): (Boolean, Int) = { def createAclsForResourceIfNotExists(resource: ResourcePattern, aclsSet: Set[AclEntry]): (Boolean, Int) = {
def create(aclData: Array[Byte]): CreateResponse = { def create(aclData: Array[Byte]): CreateResponse = {
val path = ResourceZNode.path(resource) val path = ResourceZNode.path(resource)
val createRequest = CreateRequest(path, aclData, defaultAcls(path), CreateMode.PERSISTENT) val createRequest = CreateRequest(path, aclData, defaultAcls(path), CreateMode.PERSISTENT)
@ -1173,7 +1176,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* Creates an Acl change notification message. * Creates an Acl change notification message.
* @param resource resource pattern that has changed * @param resource resource pattern that has changed
*/ */
def createAclChangeNotification(resource: Resource): Unit = { def createAclChangeNotification(resource: ResourcePattern): Unit = {
val aclChange = ZkAclStore(resource.patternType).changeStore.createChangeNode(resource) val aclChange = ZkAclStore(resource.patternType).changeStore.createChangeNode(resource)
val createRequest = CreateRequest(aclChange.path, aclChange.bytes, defaultAcls(aclChange.path), CreateMode.PERSISTENT_SEQUENTIAL) val createRequest = CreateRequest(aclChange.path, aclChange.bytes, defaultAcls(aclChange.path), CreateMode.PERSISTENT_SEQUENTIAL)
val createResponse = retryRequestUntilConnected(createRequest) val createResponse = retryRequestUntilConnected(createRequest)
@ -1251,7 +1254,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param resource * @param resource
* @return delete status * @return delete status
*/ */
def deleteResource(resource: Resource): Boolean = { def deleteResource(resource: ResourcePattern): Boolean = {
deleteRecursive(ResourceZNode.path(resource)) deleteRecursive(ResourceZNode.path(resource))
} }
@ -1260,7 +1263,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param resource * @param resource
* @return existence status * @return existence status
*/ */
def resourceExists(resource: Resource): Boolean = { def resourceExists(resource: ResourcePattern): Boolean = {
pathExists(ResourceZNode.path(resource)) pathExists(ResourceZNode.path(resource))
} }
@ -1270,7 +1273,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param expectedVersion * @param expectedVersion
* @return return true if it succeeds, false otherwise (the current version is not the expected version) * @return return true if it succeeds, false otherwise (the current version is not the expected version)
*/ */
def conditionalDelete(resource: Resource, expectedVersion: Int): Boolean = { def conditionalDelete(resource: ResourcePattern, expectedVersion: Int): Boolean = {
val deleteRequest = DeleteRequest(ResourceZNode.path(resource), expectedVersion) val deleteRequest = DeleteRequest(ResourceZNode.path(resource), expectedVersion)
val deleteResponse = retryRequestUntilConnected(deleteRequest) val deleteResponse = retryRequestUntilConnected(deleteRequest)
deleteResponse.resultCode match { deleteResponse.resultCode match {

View File

@ -26,19 +26,18 @@ import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr}
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener}
import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.security.auth.Resource.Separator
import kafka.security.authorizer.AclAuthorizer.VersionedAcls import kafka.security.authorizer.AclAuthorizer.VersionedAcls
import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.security.authorizer.AclEntry
import kafka.server.{ConfigType, DelegationTokenManager} import kafka.server.{ConfigType, DelegationTokenManager}
import kafka.utils.Json import kafka.utils.Json
import kafka.utils.json.JsonObject import kafka.utils.json.JsonObject
import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.errors.UnsupportedVersionException
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.{SecurityUtils, Time}
import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.ZooDefs
import org.apache.zookeeper.data.{ACL, Stat} import org.apache.zookeeper.data.{ACL, Stat}
@ -526,9 +525,9 @@ sealed trait ZkAclStore {
val patternType: PatternType val patternType: PatternType
val aclPath: String val aclPath: String
def path(resourceType: ResourceType): String = s"$aclPath/$resourceType" def path(resourceType: ResourceType): String = s"$aclPath/${SecurityUtils.resourceTypeName(resourceType)}"
def path(resourceType: ResourceType, resourceName: String): String = s"$aclPath/$resourceType/$resourceName" def path(resourceType: ResourceType, resourceName: String): String = s"$aclPath/${SecurityUtils.resourceTypeName(resourceType)}/$resourceName"
def changeStore: ZkAclChangeStore def changeStore: ZkAclChangeStore
} }
@ -580,7 +579,7 @@ object ExtendedAclZNode {
} }
trait AclChangeNotificationHandler { trait AclChangeNotificationHandler {
def processNotification(resource: Resource): Unit def processNotification(resource: ResourcePattern): Unit
} }
trait AclChangeSubscription extends AutoCloseable { trait AclChangeSubscription extends AutoCloseable {
@ -593,11 +592,11 @@ sealed trait ZkAclChangeStore {
val aclChangePath: String val aclChangePath: String
def createPath: String = s"$aclChangePath/${ZkAclChangeStore.SequenceNumberPrefix}" def createPath: String = s"$aclChangePath/${ZkAclChangeStore.SequenceNumberPrefix}"
def decode(bytes: Array[Byte]): Resource def decode(bytes: Array[Byte]): ResourcePattern
protected def encode(resource: Resource): Array[Byte] protected def encode(resource: ResourcePattern): Array[Byte]
def createChangeNode(resource: Resource): AclChangeNode = AclChangeNode(createPath, encode(resource)) def createChangeNode(resource: ResourcePattern): AclChangeNode = AclChangeNode(createPath, encode(resource))
def createListener(handler: AclChangeNotificationHandler, zkClient: KafkaZkClient): AclChangeSubscription = { def createListener(handler: AclChangeNotificationHandler, zkClient: KafkaZkClient): AclChangeSubscription = {
val rawHandler: NotificationHandler = (bytes: Array[Byte]) => handler.processNotification(decode(bytes)) val rawHandler: NotificationHandler = (bytes: Array[Byte]) => handler.processNotification(decode(bytes))
@ -621,18 +620,18 @@ case object LiteralAclChangeStore extends ZkAclChangeStore {
val name = "LiteralAclChangeStore" val name = "LiteralAclChangeStore"
val aclChangePath: String = "/kafka-acl-changes" val aclChangePath: String = "/kafka-acl-changes"
def encode(resource: Resource): Array[Byte] = { def encode(resource: ResourcePattern): Array[Byte] = {
if (resource.patternType != PatternType.LITERAL) if (resource.patternType != PatternType.LITERAL)
throw new IllegalArgumentException("Only literal resource patterns can be encoded") throw new IllegalArgumentException("Only literal resource patterns can be encoded")
val legacyName = resource.resourceType + Resource.Separator + resource.name val legacyName = resource.resourceType + AclEntry.ResourceSeparator + resource.name
legacyName.getBytes(UTF_8) legacyName.getBytes(UTF_8)
} }
def decode(bytes: Array[Byte]): Resource = { def decode(bytes: Array[Byte]): ResourcePattern = {
val string = new String(bytes, UTF_8) val string = new String(bytes, UTF_8)
string.split(Separator, 2) match { string.split(AclEntry.ResourceSeparator, 2) match {
case Array(resourceType, resourceName, _*) => new Resource(ResourceType.fromString(resourceType), resourceName, PatternType.LITERAL) case Array(resourceType, resourceName, _*) => new ResourcePattern(ResourceType.fromString(resourceType), resourceName, PatternType.LITERAL)
case _ => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + string) case _ => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + string)
} }
} }
@ -642,7 +641,7 @@ case object ExtendedAclChangeStore extends ZkAclChangeStore {
val name = "ExtendedAclChangeStore" val name = "ExtendedAclChangeStore"
val aclChangePath: String = "/kafka-acl-extended-changes" val aclChangePath: String = "/kafka-acl-extended-changes"
def encode(resource: Resource): Array[Byte] = { def encode(resource: ResourcePattern): Array[Byte] = {
if (resource.patternType == PatternType.LITERAL) if (resource.patternType == PatternType.LITERAL)
throw new IllegalArgumentException("Literal pattern types are not supported") throw new IllegalArgumentException("Literal pattern types are not supported")
@ -653,7 +652,7 @@ case object ExtendedAclChangeStore extends ZkAclChangeStore {
resource.patternType.name)) resource.patternType.name))
} }
def decode(bytes: Array[Byte]): Resource = { def decode(bytes: Array[Byte]): ResourcePattern = {
val changeEvent = Json.parseBytesAs[ExtendedAclChangeEvent](bytes) match { val changeEvent = Json.parseBytesAs[ExtendedAclChangeEvent](bytes) match {
case Right(event) => event case Right(event) => event
case Left(e) => throw new IllegalArgumentException("Failed to parse ACL change event", e) case Left(e) => throw new IllegalArgumentException("Failed to parse ACL change event", e)
@ -667,10 +666,10 @@ case object ExtendedAclChangeStore extends ZkAclChangeStore {
} }
object ResourceZNode { object ResourceZNode {
def path(resource: Resource): String = ZkAclStore(resource.patternType).path(resource.resourceType, resource.name) def path(resource: ResourcePattern): String = ZkAclStore(resource.patternType).path(resource.resourceType, resource.name)
def encode(acls: Set[Acl]): Array[Byte] = Json.encodeAsBytes(Acl.toJsonCompatibleMap(acls).asJava) def encode(acls: Set[AclEntry]): Array[Byte] = Json.encodeAsBytes(AclEntry.toJsonCompatibleMap(acls).asJava)
def decode(bytes: Array[Byte], stat: Stat): VersionedAcls = VersionedAcls(Acl.fromBytes(bytes), stat.getVersion) def decode(bytes: Array[Byte], stat: Stat): VersionedAcls = VersionedAcls(AclEntry.fromBytes(bytes), stat.getVersion)
} }
object ExtendedAclChangeEvent { object ExtendedAclChangeEvent {
@ -684,11 +683,11 @@ case class ExtendedAclChangeEvent(@BeanProperty @JsonProperty("version") version
if (version > ExtendedAclChangeEvent.currentVersion) if (version > ExtendedAclChangeEvent.currentVersion)
throw new UnsupportedVersionException(s"Acl change event received for unsupported version: $version") throw new UnsupportedVersionException(s"Acl change event received for unsupported version: $version")
def toResource: Try[Resource] = { def toResource: Try[ResourcePattern] = {
for { for {
resType <- Try(ResourceType.fromString(resourceType)) resType <- Try(ResourceType.fromString(resourceType))
patType <- Try(PatternType.fromString(patternType)) patType <- Try(PatternType.fromString(patternType))
resource = Resource(resType, name, patType) resource = new ResourcePattern(resType, name, patType)
} yield resource } yield resource
} }
} }

View File

@ -20,8 +20,8 @@ import java.util.{Collections, Optional, Properties}
import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.security.auth.{SimpleAclAuthorizer, Topic, ResourceType => AuthResourceType} import kafka.security.authorizer.AclEntry
import kafka.security.authorizer.AuthorizerUtils.WildcardHost import kafka.security.authorizer.AclEntry.WildcardHost
import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.server.{BaseRequestTest, KafkaConfig}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp}
@ -117,7 +117,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group)
override def brokerPropertyOverrides(properties: Properties): Unit = { override def brokerPropertyOverrides(properties: Properties): Unit = {
properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[SimpleAclAuthorizer].getName) properties.put(KafkaConfig.AuthorizerClassNameProp, "kafka.security.auth.SimpleAclAuthorizer")
properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
@ -1663,11 +1663,11 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
val authorizationErrors = resources.flatMap { resourceType => val authorizationErrors = resources.flatMap { resourceType =>
if (resourceType == TOPIC) { if (resourceType == TOPIC) {
if (isAuthorized) if (isAuthorized)
Set(Errors.UNKNOWN_TOPIC_OR_PARTITION, Topic.error) Set(Errors.UNKNOWN_TOPIC_OR_PARTITION, AclEntry.authorizationError(ResourceType.TOPIC))
else else
Set(Topic.error) Set(AclEntry.authorizationError(ResourceType.TOPIC))
} else { } else {
Set(AuthResourceType.fromJava(resourceType).error) Set(AclEntry.authorizationError(resourceType))
} }
} }

View File

@ -20,13 +20,14 @@ import java.util
import java.util.Properties import java.util.Properties
import java.util.concurrent.ExecutionException import java.util.concurrent.ExecutionException
import kafka.security.auth.{Cluster, Topic} import kafka.security.authorizer.AclEntry
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.Logging import kafka.utils.Logging
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription}
import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException} import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.resource.ResourceType
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.junit.Assert._ import org.junit.Assert._
import org.junit.rules.Timeout import org.junit.rules.Timeout
@ -176,13 +177,12 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
//with includeAuthorizedOperations flag //with includeAuthorizedOperations flag
topicResult = getTopicMetadata(client, topic, new DescribeTopicsOptions().includeAuthorizedOperations(true)) topicResult = getTopicMetadata(client, topic, new DescribeTopicsOptions().includeAuthorizedOperations(true))
expectedOperations = Topic.supportedOperations expectedOperations = AclEntry.supportedOperations(ResourceType.TOPIC).asJava
.map(operation => operation.toJava).asJava
assertEquals(expectedOperations, topicResult.authorizedOperations) assertEquals(expectedOperations, topicResult.authorizedOperations)
} }
def configuredClusterPermissions(): Set[AclOperation] = { def configuredClusterPermissions(): Set[AclOperation] = {
Cluster.supportedOperations.map(operation => operation.toJava) AclEntry.supportedOperations(ResourceType.CLUSTER)
} }
override def modifyConfigs(configs: Seq[Properties]): Unit = { override def modifyConfigs(configs: Seq[Properties]): Unit = {

View File

@ -16,15 +16,14 @@ import java.io.File
import java.util import java.util
import java.util.Properties import java.util.Properties
import kafka.security.auth.{Cluster, Group, Resource, Topic, Acl => AuthAcl} import kafka.security.authorizer.{AclAuthorizer, AclEntry}
import kafka.security.authorizer.AclAuthorizer
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils}
import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.admin._
import org.apache.kafka.common.acl.AclOperation.{ALTER, CLUSTER_ACTION, DESCRIBE} import org.apache.kafka.common.acl.AclOperation.{ALTER, CLUSTER_ACTION, DESCRIBE}
import org.apache.kafka.common.acl.AclPermissionType.ALLOW import org.apache.kafka.common.acl.AclPermissionType.ALLOW
import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl._
import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
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
@ -51,8 +50,8 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
override def configureSecurityBeforeServersStart(): Unit = { override def configureSecurityBeforeServersStart(): Unit = {
val authorizer = CoreUtils.createObject[Authorizer](classOf[AclAuthorizer].getName) val authorizer = CoreUtils.createObject[Authorizer](classOf[AclAuthorizer].getName)
val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.ClusterResource.name, PatternType.LITERAL) val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL)
val topicResource = new ResourcePattern(ResourceType.TOPIC, Resource.WildCardResource, PatternType.LITERAL) val topicResource = new ResourcePattern(ResourceType.TOPIC, AclEntry.WildcardResource, PatternType.LITERAL)
try { try {
authorizer.configure(this.configs.head.originals()) authorizer.configure(this.configs.head.originals())
@ -76,7 +75,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
private def accessControlEntry(userName: String, permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = { private def accessControlEntry(userName: String, permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = {
new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, userName).toString, new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, userName).toString,
AuthAcl.WildCardHost, operation, permissionType) AclEntry.WildcardHost, operation, permissionType)
} }
@After @After
@ -96,7 +95,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
val group3Acl = new AclBinding(new ResourcePattern(ResourceType.GROUP, group3, PatternType.LITERAL), val group3Acl = new AclBinding(new ResourcePattern(ResourceType.GROUP, group3, PatternType.LITERAL),
new AccessControlEntry("User:" + JaasTestUtils.KafkaClientPrincipalUnqualifiedName2, "*", AclOperation.DELETE, AclPermissionType.ALLOW)) new AccessControlEntry("User:" + JaasTestUtils.KafkaClientPrincipalUnqualifiedName2, "*", AclOperation.DELETE, AclPermissionType.ALLOW))
val clusterAllAcl = new AclBinding(Resource.ClusterResource.toPattern, val clusterAllAcl = new AclBinding(new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL),
new AccessControlEntry("User:" + JaasTestUtils.KafkaClientPrincipalUnqualifiedName2, "*", AclOperation.ALL, AclPermissionType.ALLOW)) new AccessControlEntry("User:" + JaasTestUtils.KafkaClientPrincipalUnqualifiedName2, "*", AclOperation.ALL, AclPermissionType.ALLOW))
val topic1Acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic1, PatternType.LITERAL), val topic1Acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic1, PatternType.LITERAL),
@ -126,8 +125,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
val describeConsumerGroupsResult = client.describeConsumerGroups(Seq(group1, group2, group3).asJava, val describeConsumerGroupsResult = client.describeConsumerGroups(Seq(group1, group2, group3).asJava,
new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true)) new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true))
assertEquals(3, describeConsumerGroupsResult.describedGroups().size()) assertEquals(3, describeConsumerGroupsResult.describedGroups().size())
val expectedOperations = Group.supportedOperations val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP).asJava
.map(operation => operation.toJava).asJava
val group1Description = describeConsumerGroupsResult.describedGroups().get(group1).get val group1Description = describeConsumerGroupsResult.describedGroups().get(group1).get
assertEquals(expectedOperations, group1Description.authorizedOperations()) assertEquals(expectedOperations, group1Description.authorizedOperations())
@ -159,8 +157,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
assertEquals(Set(clusterAllAcl), results.values.keySet.asScala) assertEquals(Set(clusterAllAcl), results.values.keySet.asScala)
results.all.get results.all.get
val expectedOperations = Cluster.supportedOperations val expectedOperations = AclEntry.supportedOperations(ResourceType.CLUSTER).asJava
.map(operation => operation.toJava).asJava
clusterDescribeResult = client.describeCluster(new DescribeClusterOptions(). clusterDescribeResult = client.describeCluster(new DescribeClusterOptions().
includeAuthorizedOperations(true)) includeAuthorizedOperations(true))
@ -189,8 +186,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
assertEquals(Set(topic1Acl, topic2All), results.values.keySet.asScala) assertEquals(Set(topic1Acl, topic2All), results.values.keySet.asScala)
results.all.get results.all.get
val expectedOperations = Topic.supportedOperations val expectedOperations = AclEntry.supportedOperations(ResourceType.TOPIC).asJava
.map(operation => operation.toJava).asJava
describeTopicsResult = client.describeTopics(Set(topic1, topic2).asJava, describeTopicsResult = client.describeTopics(Set(topic1, topic2).asJava,
new DescribeTopicsOptions().includeAuthorizedOperations(true)).all.get() new DescribeTopicsOptions().includeAuthorizedOperations(true)).all.get()

View File

@ -25,7 +25,7 @@ import java.util.concurrent.ExecutionException
import kafka.admin.AclCommand import kafka.admin.AclCommand
import kafka.security.authorizer.AclAuthorizer import kafka.security.authorizer.AclAuthorizer
import kafka.security.authorizer.AuthorizerUtils.WildcardHost import kafka.security.authorizer.AclEntry.WildcardHost
import kafka.server._ import kafka.server._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords}

View File

@ -26,7 +26,7 @@ import java.util.{Collections, Optional, Properties}
import java.{time, util} import java.{time, util}
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.security.auth.Group import kafka.security.authorizer.AclEntry
import kafka.server.{Defaults, KafkaConfig, KafkaServer} import kafka.server.{Defaults, KafkaConfig, KafkaServer}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.utils.{Log4jController, TestUtils} import kafka.utils.{Log4jController, TestUtils}
@ -1077,8 +1077,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals(testNumPartitions, topicPartitions.size()) assertEquals(testNumPartitions, topicPartitions.size())
assertEquals(testNumPartitions, topicPartitions.asScala. assertEquals(testNumPartitions, topicPartitions.asScala.
count(tp => tp.topic().equals(testTopicName))) count(tp => tp.topic().equals(testTopicName)))
val expectedOperations = Group.supportedOperations val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP).asJava
.map(operation => operation.toJava).asJava
assertEquals(expectedOperations, testGroupDescription.authorizedOperations()) assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
// Test that the fake group is listed as dead. // Test that the fake group is listed as dead.

View File

@ -22,6 +22,8 @@ import kafka.utils.JaasTestUtils
import scala.collection.immutable.List import scala.collection.immutable.List
// Note: this test currently uses the deprecated SimpleAclAuthorizer to ensure we have test coverage
// It must be replaced with the new AclAuthorizer when SimpleAclAuthorizer is removed
class SaslGssapiSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest { class SaslGssapiSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest {
override val clientPrincipal = JaasTestUtils.KafkaClientPrincipalUnqualifiedName override val clientPrincipal = JaasTestUtils.KafkaClientPrincipalUnqualifiedName
override val kafkaPrincipal = JaasTestUtils.KafkaServerPrincipalUnqualifiedName override val kafkaPrincipal = JaasTestUtils.KafkaServerPrincipalUnqualifiedName

View File

@ -16,13 +16,13 @@ import java.io.File
import java.util import java.util
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.security.auth.{All, Allow, Alter, AlterConfigs, Authorizer, ClusterAction, Create, Delete, Deny, Describe, Group, Operation, PermissionType, SimpleAclAuthorizer, Topic, Acl => AuthAcl, Resource => AuthResource}
import kafka.security.authorizer.AuthorizerWrapper
import kafka.server.{Defaults, KafkaConfig} import kafka.server.{Defaults, KafkaConfig}
import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.admin._
import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl._
import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE}
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType}
@ -36,32 +36,29 @@ import scala.compat.java8.OptionConverters._
import scala.concurrent.ExecutionException import scala.concurrent.ExecutionException
import scala.util.{Failure, Success, Try} import scala.util.{Failure, Success, Try}
abstract class AuthorizationAdmin {
def authorizerClassName: String
def initializeAcls(): Unit
def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit
def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit
}
// Note: this test currently uses the deprecated SimpleAclAuthorizer to ensure we have test coverage
// It must be replaced with the new AclAuthorizer when SimpleAclAuthorizer is removed
class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup { class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup {
val authorizationAdmin: AuthorizationAdmin = new LegacyAuthorizationAdmin
this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true")
// This tests the old SimpleAclAuthorizer, we have another test for the new AclAuthorizer
this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[SimpleAclAuthorizer].getName)
override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected def securityProtocol = SecurityProtocol.SASL_SSL
override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
override def configureSecurityBeforeServersStart(): Unit = { override def generateConfigs: Seq[KafkaConfig] = {
val authorizer = CoreUtils.createObject[Authorizer](classOf[SimpleAclAuthorizer].getName) this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizationAdmin.authorizerClassName)
try { super.generateConfigs
authorizer.configure(this.configs.head.originals()) }
authorizer.addAcls(Set(new AuthAcl(AuthAcl.WildCardPrincipal, Allow,
AuthAcl.WildCardHost, All)), new AuthResource(Topic, "*", PatternType.LITERAL))
authorizer.addAcls(Set(new AuthAcl(AuthAcl.WildCardPrincipal, Allow,
AuthAcl.WildCardHost, All)), new AuthResource(Group, "*", PatternType.LITERAL))
authorizer.addAcls(Set(clusterAcl(Allow, Create), override def configureSecurityBeforeServersStart(): Unit = {
clusterAcl(Allow, Delete), authorizationAdmin.initializeAcls()
clusterAcl(Allow, ClusterAction),
clusterAcl(Allow, AlterConfigs),
clusterAcl(Allow, Alter)),
AuthResource.ClusterResource)
} finally {
authorizer.close()
}
} }
@Before @Before
@ -74,27 +71,6 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KafkaServerContextName)) startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KafkaServerContextName))
} }
private def clusterAcl(permissionType: PermissionType, operation: Operation): AuthAcl = {
new AuthAcl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*"), permissionType,
AuthAcl.WildCardHost, operation)
}
def addClusterAcl(permissionType: PermissionType, operation: Operation): Unit = {
val acls = Set(clusterAcl(permissionType, operation))
val authorizer = simpleAclAuthorizer
val prevAcls = authorizer.getAcls(AuthResource.ClusterResource)
authorizer.addAcls(acls, AuthResource.ClusterResource)
TestUtils.waitAndVerifyAcls(prevAcls ++ acls, authorizer, AuthResource.ClusterResource)
}
def removeClusterAcl(permissionType: PermissionType, operation: Operation): Unit = {
val acls = Set(clusterAcl(permissionType, operation))
val authorizer = simpleAclAuthorizer
val prevAcls = authorizer.getAcls(AuthResource.ClusterResource)
Assert.assertTrue(authorizer.removeAcls(acls, AuthResource.ClusterResource))
TestUtils.waitAndVerifyAcls(prevAcls -- acls, authorizer, AuthResource.ClusterResource)
}
@After @After
override def tearDown(): Unit = { override def tearDown(): Unit = {
super.tearDown() super.tearDown()
@ -380,30 +356,30 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
def testAclAuthorizationDenied(): Unit = { def testAclAuthorizationDenied(): Unit = {
client = Admin.create(createConfig()) client = Admin.create(createConfig())
// Test that we cannot create or delete ACLs when Alter is denied. // Test that we cannot create or delete ACLs when ALTER is denied.
addClusterAcl(Deny, Alter) authorizationAdmin.addClusterAcl(DENY, ALTER)
testAclGet(expectAuth = true) testAclGet(expectAuth = true)
testAclCreateGetDelete(expectAuth = false) testAclCreateGetDelete(expectAuth = false)
// Test that we cannot do anything with ACLs when Describe and Alter are denied. // Test that we cannot do anything with ACLs when DESCRIBE and ALTER are denied.
addClusterAcl(Deny, Describe) authorizationAdmin.addClusterAcl(DENY, DESCRIBE)
testAclGet(expectAuth = false) testAclGet(expectAuth = false)
testAclCreateGetDelete(expectAuth = false) testAclCreateGetDelete(expectAuth = false)
// Test that we can create, delete, and get ACLs with the default ACLs. // Test that we can create, delete, and get ACLs with the default ACLs.
removeClusterAcl(Deny, Describe) authorizationAdmin.removeClusterAcl(DENY, DESCRIBE)
removeClusterAcl(Deny, Alter) authorizationAdmin.removeClusterAcl(DENY, ALTER)
testAclGet(expectAuth = true) testAclGet(expectAuth = true)
testAclCreateGetDelete(expectAuth = true) testAclCreateGetDelete(expectAuth = true)
// Test that we can't do anything with ACLs without the Allow Alter ACL in place. // Test that we can't do anything with ACLs without the ALLOW ALTER ACL in place.
removeClusterAcl(Allow, Alter) authorizationAdmin.removeClusterAcl(ALLOW, ALTER)
removeClusterAcl(Allow, Delete) authorizationAdmin.removeClusterAcl(ALLOW, DELETE)
testAclGet(expectAuth = false) testAclGet(expectAuth = false)
testAclCreateGetDelete(expectAuth = false) testAclCreateGetDelete(expectAuth = false)
// Test that we can describe, but not alter ACLs, with only the Allow Describe ACL in place. // Test that we can describe, but not alter ACLs, with only the ALLOW DESCRIBE ACL in place.
addClusterAcl(Allow, Describe) authorizationAdmin.addClusterAcl(ALLOW, DESCRIBE)
testAclGet(expectAuth = true) testAclGet(expectAuth = true)
testAclCreateGetDelete(expectAuth = false) testAclCreateGetDelete(expectAuth = false)
} }
@ -498,8 +474,58 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
client.describeAcls(allTopicAcls).values.get().asScala.toSet client.describeAcls(allTopicAcls).values.get().asScala.toSet
} }
private def simpleAclAuthorizer: Authorizer = { @deprecated("Use kafka.security.authorizer.AclAuthorizer", "Since 2.5")
val authorizerWrapper = servers.head.dataPlaneRequestProcessor.authorizer.get.asInstanceOf[AuthorizerWrapper] class LegacyAuthorizationAdmin extends AuthorizationAdmin {
authorizerWrapper.baseAuthorizer import kafka.security.auth._
import kafka.security.authorizer.AuthorizerWrapper
override def authorizerClassName: String = classOf[SimpleAclAuthorizer].getName
override def initializeAcls(): Unit = {
val authorizer = CoreUtils.createObject[Authorizer](classOf[SimpleAclAuthorizer].getName)
try {
authorizer.configure(configs.head.originals())
authorizer.addAcls(Set(new Acl(Acl.WildCardPrincipal, Allow,
Acl.WildCardHost, All)), new Resource(Topic, "*", PatternType.LITERAL))
authorizer.addAcls(Set(new Acl(Acl.WildCardPrincipal, Allow,
Acl.WildCardHost, All)), new Resource(Group, "*", PatternType.LITERAL))
authorizer.addAcls(Set(clusterAcl(ALLOW, CREATE),
clusterAcl(ALLOW, DELETE),
clusterAcl(ALLOW, CLUSTER_ACTION),
clusterAcl(ALLOW, ALTER_CONFIGS),
clusterAcl(ALLOW, ALTER)),
Resource.ClusterResource)
} finally {
authorizer.close()
}
}
override def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = {
val acls = Set(clusterAcl(permissionType, operation))
val authorizer = simpleAclAuthorizer
val prevAcls = authorizer.getAcls(Resource.ClusterResource)
authorizer.addAcls(acls, Resource.ClusterResource)
TestUtils.waitAndVerifyAcls(prevAcls ++ acls, authorizer, Resource.ClusterResource)
}
override def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = {
val acls = Set(clusterAcl(permissionType, operation))
val authorizer = simpleAclAuthorizer
val prevAcls = authorizer.getAcls(Resource.ClusterResource)
Assert.assertTrue(authorizer.removeAcls(acls, Resource.ClusterResource))
TestUtils.waitAndVerifyAcls(prevAcls -- acls, authorizer, Resource.ClusterResource)
}
private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): Acl = {
new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*"), PermissionType.fromJava(permissionType),
Acl.WildCardHost, Operation.fromJava(operation))
}
private def simpleAclAuthorizer: Authorizer = {
val authorizerWrapper = servers.head.dataPlaneRequestProcessor.authorizer.get.asInstanceOf[AuthorizerWrapper]
authorizerWrapper.baseAuthorizer
}
} }
} }

View File

@ -20,8 +20,7 @@ import java.util.concurrent._
import com.yammer.metrics.Metrics import com.yammer.metrics.Metrics
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import kafka.security.authorizer.AclAuthorizer import kafka.security.authorizer.AclAuthorizer
import kafka.security.authorizer.AuthorizerUtils.{WildcardHost, WildcardPrincipal} import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString}
import kafka.security.auth.{Operation, PermissionType}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.{CoreUtils, TestUtils} import kafka.utils.{CoreUtils, TestUtils}
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateAclsResult} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateAclsResult}
@ -86,34 +85,15 @@ object SslAdminIntegrationTest {
} }
class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
override val authorizationAdmin = new AclAuthorizationAdmin
val clusterResourcePattern = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) val clusterResourcePattern = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL)
this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true")
this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[SslAdminIntegrationTest.TestableAclAuthorizer].getName)
override protected def securityProtocol = SecurityProtocol.SSL override protected def securityProtocol = SecurityProtocol.SSL
override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
private val adminClients = mutable.Buffer.empty[Admin] private val adminClients = mutable.Buffer.empty[Admin]
override def configureSecurityBeforeServersStart(): Unit = {
val authorizer = CoreUtils.createObject[Authorizer](classOf[AclAuthorizer].getName)
try {
authorizer.configure(this.configs.head.originals())
val ace = new AccessControlEntry(WildcardPrincipal, WildcardHost, ALL, ALLOW)
authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava)
authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava)
authorizer.createAcls(null, List(clusterAcl(ALLOW, CREATE),
clusterAcl(ALLOW, DELETE),
clusterAcl(ALLOW, CLUSTER_ACTION),
clusterAcl(ALLOW, ALTER_CONFIGS),
clusterAcl(ALLOW, ALTER))
.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava)
} finally {
authorizer.close()
}
}
override def setUpSasl(): Unit = { override def setUpSasl(): Unit = {
SslAdminIntegrationTest.semaphore = None SslAdminIntegrationTest.semaphore = None
SslAdminIntegrationTest.executor = None SslAdminIntegrationTest.executor = None
@ -132,32 +112,6 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
super.tearDown() super.tearDown()
} }
override def addClusterAcl(permissionType: PermissionType, operation: Operation): Unit = {
val ace = clusterAcl(permissionType.toJava, operation.toJava)
val aclBinding = new AclBinding(clusterResourcePattern, ace)
val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get
val prevAcls = authorizer.acls(new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY))
.asScala.map(_.entry).toSet
authorizer.createAcls(null, Collections.singletonList(aclBinding))
TestUtils.waitAndVerifyAcls(prevAcls ++ Set(ace), authorizer, clusterResourcePattern)
}
override def removeClusterAcl(permissionType: PermissionType, operation: Operation): Unit = {
val ace = clusterAcl(permissionType.toJava, operation.toJava)
val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get
val clusterFilter = new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY)
val prevAcls = authorizer.acls(clusterFilter).asScala.map(_.entry).toSet
val deleteFilter = new AclBindingFilter(clusterResourcePattern.toFilter, ace.toFilter)
Assert.assertFalse(authorizer.deleteAcls(null, Collections.singletonList(deleteFilter))
.get(0).toCompletableFuture.get.aclBindingDeleteResults().asScala.head.exception.isPresent)
TestUtils.waitAndVerifyAcls(prevAcls -- Set(ace), authorizer, clusterResourcePattern)
}
private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = {
new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString,
WildcardHost, operation, permissionType)
}
@Test @Test
def testAclUpdatesUsingSynchronousAuthorizer(): Unit = { def testAclUpdatesUsingSynchronousAuthorizer(): Unit = {
verifyAclUpdates() verifyAclUpdates()
@ -312,4 +266,54 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
assertTrue(s"Unable to find metric $name: allMetrics: ${allMetrics.keySet.map(_.getMBeanName)}", metrics.nonEmpty) assertTrue(s"Unable to find metric $name: allMetrics: ${allMetrics.keySet.map(_.getMBeanName)}", metrics.nonEmpty)
metrics.map(_.asInstanceOf[Gauge[Int]].value).sum metrics.map(_.asInstanceOf[Gauge[Int]].value).sum
} }
class AclAuthorizationAdmin extends AuthorizationAdmin {
override def authorizerClassName: String = classOf[SslAdminIntegrationTest.TestableAclAuthorizer].getName
override def initializeAcls(): Unit = {
val authorizer = CoreUtils.createObject[Authorizer](classOf[AclAuthorizer].getName)
try {
authorizer.configure(configs.head.originals())
val ace = new AccessControlEntry(WildcardPrincipalString, WildcardHost, ALL, ALLOW)
authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava)
authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava)
authorizer.createAcls(null, List(clusterAcl(ALLOW, CREATE),
clusterAcl(ALLOW, DELETE),
clusterAcl(ALLOW, CLUSTER_ACTION),
clusterAcl(ALLOW, ALTER_CONFIGS),
clusterAcl(ALLOW, ALTER))
.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava)
} finally {
authorizer.close()
}
}
override def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = {
val ace = clusterAcl(permissionType, operation)
val aclBinding = new AclBinding(clusterResourcePattern, ace)
val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get
val prevAcls = authorizer.acls(new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY))
.asScala.map(_.entry).toSet
authorizer.createAcls(null, Collections.singletonList(aclBinding))
TestUtils.waitAndVerifyAcls(prevAcls ++ Set(ace), authorizer, clusterResourcePattern)
}
override def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = {
val ace = clusterAcl(permissionType, operation)
val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get
val clusterFilter = new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY)
val prevAcls = authorizer.acls(clusterFilter).asScala.map(_.entry).toSet
val deleteFilter = new AclBindingFilter(clusterResourcePattern.toFilter, ace.toFilter)
Assert.assertFalse(authorizer.deleteAcls(null, Collections.singletonList(deleteFilter))
.get(0).toCompletableFuture.get.aclBindingDeleteResults().asScala.head.exception.isPresent)
TestUtils.waitAndVerifyAcls(prevAcls -- Set(ace), authorizer, clusterResourcePattern)
}
private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = {
new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString,
WildcardHost, operation, permissionType)
}
}
} }

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.junit.Test import org.junit.Test
import org.junit.Assert._ import org.junit.Assert._
@deprecated("Use org.apache.kafka.common.resource.ResourcePattern", "Since 2.5")
class ResourceTest { class ResourceTest {
@Test(expected = classOf[KafkaException]) @Test(expected = classOf[KafkaException])
def shouldThrowOnTwoPartStringWithUnknownResourceType(): Unit = { def shouldThrowOnTwoPartStringWithUnknownResourceType(): Unit = {

View File

@ -17,20 +17,21 @@
package kafka.zk package kafka.zk
import kafka.security.auth.{Resource, Topic}
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.resource.ResourceType.TOPIC
import org.junit.Assert.assertEquals import org.junit.Assert.assertEquals
import org.junit.Test import org.junit.Test
class ExtendedAclStoreTest { class ExtendedAclStoreTest {
private val literalResource = Resource(Topic, "some-topic", LITERAL) private val literalResource = new ResourcePattern(TOPIC, "some-topic", LITERAL)
private val prefixedResource = Resource(Topic, "some-topic", PREFIXED) private val prefixedResource = new ResourcePattern(TOPIC, "some-topic", PREFIXED)
private val store = new ExtendedAclStore(PREFIXED) private val store = new ExtendedAclStore(PREFIXED)
@Test @Test
def shouldHaveCorrectPaths(): Unit = { def shouldHaveCorrectPaths(): Unit = {
assertEquals("/kafka-acl-extended/prefixed", store.aclPath) assertEquals("/kafka-acl-extended/prefixed", store.aclPath)
assertEquals("/kafka-acl-extended/prefixed/Topic", store.path(Topic)) assertEquals("/kafka-acl-extended/prefixed/Topic", store.path(TOPIC))
assertEquals("/kafka-acl-extended-changes", store.changeStore.aclChangePath) assertEquals("/kafka-acl-extended-changes", store.changeStore.aclChangePath)
} }

View File

@ -19,20 +19,22 @@ package kafka.zk
import java.nio.charset.StandardCharsets.UTF_8 import java.nio.charset.StandardCharsets.UTF_8
import kafka.security.auth.{Group, Resource, Topic} import kafka.security.authorizer.AclEntry
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC}
import org.junit.Assert.assertEquals import org.junit.Assert.assertEquals
import org.junit.Test import org.junit.Test
class LiteralAclStoreTest { class LiteralAclStoreTest {
private val literalResource = Resource(Topic, "some-topic", LITERAL) private val literalResource = new ResourcePattern(TOPIC, "some-topic", LITERAL)
private val prefixedResource = Resource(Topic, "some-topic", PREFIXED) private val prefixedResource = new ResourcePattern(TOPIC, "some-topic", PREFIXED)
private val store = LiteralAclStore private val store = LiteralAclStore
@Test @Test
def shouldHaveCorrectPaths(): Unit = { def shouldHaveCorrectPaths(): Unit = {
assertEquals("/kafka-acl", store.aclPath) assertEquals("/kafka-acl", store.aclPath)
assertEquals("/kafka-acl/Topic", store.path(Topic)) assertEquals("/kafka-acl/Topic", store.path(TOPIC))
assertEquals("/kafka-acl-changes", store.changeStore.aclChangePath) assertEquals("/kafka-acl-changes", store.changeStore.aclChangePath)
} }
@ -64,8 +66,8 @@ class LiteralAclStoreTest {
@Test @Test
def shouldDecodeResourceUsingTwoPartLogic(): Unit = { def shouldDecodeResourceUsingTwoPartLogic(): Unit = {
val resource = Resource(Group, "PREFIXED:this, including the PREFIXED part, is a valid two part group name", LITERAL) val resource = new ResourcePattern(GROUP, "PREFIXED:this, including the PREFIXED part, is a valid two part group name", LITERAL)
val encoded = (resource.resourceType + Resource.Separator + resource.name).getBytes(UTF_8) val encoded = (resource.resourceType + AclEntry.ResourceSeparator + resource.name).getBytes(UTF_8)
val actual = store.changeStore.decode(encoded) val actual = store.changeStore.decode(encoded)

View File

@ -19,8 +19,7 @@ package kafka.admin
import java.util.Properties import java.util.Properties
import kafka.admin.AclCommand.AclCommandOptions import kafka.admin.AclCommand.AclCommandOptions
import kafka.security.auth.Authorizer import kafka.security.authorizer.{AclAuthorizer, AclEntry}
import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils}
import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{Exit, Logging, TestUtils} import kafka.utils.{Exit, Logging, TestUtils}
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
@ -33,7 +32,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.SecurityUtils import org.apache.kafka.common.utils.SecurityUtils
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.Authorizer
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
@ -64,13 +63,13 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
private val ResourceToOperations = Map[Set[ResourcePattern], (Set[AclOperation], Array[String])]( private val ResourceToOperations = Map[Set[ResourcePattern], (Set[AclOperation], Array[String])](
TopicResources -> (Set(READ, WRITE, CREATE, DESCRIBE, DELETE, DESCRIBE_CONFIGS, ALTER_CONFIGS, ALTER), TopicResources -> (Set(READ, WRITE, CREATE, DESCRIBE, DELETE, DESCRIBE_CONFIGS, ALTER_CONFIGS, ALTER),
Array("--operation", "Read" , "--operation", "WRITE", "--operation", "Create", "--operation", "Describe", "--operation", "Delete", Array("--operation", "Read" , "--operation", "Write", "--operation", "Create", "--operation", "Describe", "--operation", "Delete",
"--operation", "DescribeConfigs", "--operation", "AlterConfigs", "--operation", "Alter")), "--operation", "DescribeConfigs", "--operation", "AlterConfigs", "--operation", "Alter")),
Set(ClusterResource) -> (Set(CREATE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, ALTER, DESCRIBE), Set(ClusterResource) -> (Set(CREATE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, ALTER, DESCRIBE),
Array("--operation", "Create", "--operation", "ClusterAction", "--operation", "DescribeConfigs", Array("--operation", "Create", "--operation", "ClusterAction", "--operation", "DescribeConfigs",
"--operation", "AlterConfigs", "--operation", "IdempotentWrite", "--operation", "Alter", "--operation", "Describe")), "--operation", "AlterConfigs", "--operation", "IdempotentWrite", "--operation", "Alter", "--operation", "Describe")),
GroupResources -> (Set(READ, DESCRIBE, DELETE), Array("--operation", "Read", "--operation", "Describe", "--operation", "Delete")), GroupResources -> (Set(READ, DESCRIBE, DELETE), Array("--operation", "Read", "--operation", "Describe", "--operation", "Delete")),
TransactionalIdResources -> (Set(DESCRIBE, WRITE), Array("--operation", "Describe", "--operation", "WRITE")), TransactionalIdResources -> (Set(DESCRIBE, WRITE), Array("--operation", "Describe", "--operation", "Write")),
TokenResources -> (Set(DESCRIBE), Array("--operation", "Describe")) TokenResources -> (Set(DESCRIBE), Array("--operation", "Describe"))
) )
@ -194,9 +193,9 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
AclCommand.main(cmdArgs ++ cmd :+ "--add") AclCommand.main(cmdArgs ++ cmd :+ "--add")
withAuthorizer() { authorizer => withAuthorizer() { authorizer =>
val writeAcl = new AccessControlEntry(principal.toString, AuthorizerUtils.WildcardHost, WRITE, ALLOW) val writeAcl = new AccessControlEntry(principal.toString, AclEntry.WildcardHost, WRITE, ALLOW)
val describeAcl = new AccessControlEntry(principal.toString, AuthorizerUtils.WildcardHost, DESCRIBE, ALLOW) val describeAcl = new AccessControlEntry(principal.toString, AclEntry.WildcardHost, DESCRIBE, ALLOW)
val createAcl = new AccessControlEntry(principal.toString, AuthorizerUtils.WildcardHost, CREATE, ALLOW) val createAcl = new AccessControlEntry(principal.toString, AclEntry.WildcardHost, CREATE, ALLOW)
TestUtils.waitAndVerifyAcls(Set(writeAcl, describeAcl, createAcl), authorizer, TestUtils.waitAndVerifyAcls(Set(writeAcl, describeAcl, createAcl), authorizer,
new ResourcePattern(TOPIC, "Test-", PREFIXED)) new ResourcePattern(TOPIC, "Test-", PREFIXED))
} }
@ -212,14 +211,8 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
@Test(expected = classOf[IllegalArgumentException]) @Test(expected = classOf[IllegalArgumentException])
def testInvalidAuthorizerProperty(): Unit = { def testInvalidAuthorizerProperty(): Unit = {
val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect) val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect)
val aclCommandService = new AclCommand.AuthorizerService(classOf[Authorizer], new AclCommandOptions(args)) val aclCommandService = new AclCommand.AuthorizerService(classOf[AclAuthorizer].getName,
aclCommandService.listAcls() new AclCommandOptions(args))
}
@Test(expected = classOf[IllegalArgumentException])
def testInvalidJAuthorizerProperty(): Unit = {
val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect)
val aclCommandService = new AclCommand.JAuthorizerService(classOf[JAuthorizer], new AclCommandOptions(args))
aclCommandService.listAcls() aclCommandService.listAcls()
} }
@ -273,7 +266,7 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
Users.foldLeft(cmd) ((cmd, user) => cmd ++ Array(principalCmd, user.toString)) Users.foldLeft(cmd) ((cmd, user) => cmd ++ Array(principalCmd, user.toString))
} }
private def withAuthorizer()(f: JAuthorizer => Unit): Unit = { private def withAuthorizer()(f: Authorizer => Unit): Unit = {
val kafkaConfig = KafkaConfig.fromProps(brokerProps, doLog = false) val kafkaConfig = KafkaConfig.fromProps(brokerProps, doLog = false)
val authZ = new AclAuthorizer val authZ = new AclAuthorizer
try { try {

View File

@ -16,10 +16,11 @@
*/ */
package kafka.common package kafka.common
import kafka.security.auth.{Group, Resource}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.zk.{LiteralAclChangeStore, LiteralAclStore, ZkAclChangeStore, ZooKeeperTestHarness} import kafka.zk.{LiteralAclChangeStore, LiteralAclStore, ZkAclChangeStore, ZooKeeperTestHarness}
import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.PatternType.LITERAL
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.resource.ResourceType.GROUP
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
@ -48,8 +49,8 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness {
@Test @Test
def testProcessNotification(): Unit = { def testProcessNotification(): Unit = {
val notificationMessage1 = Resource(Group, "messageA", LITERAL) val notificationMessage1 = new ResourcePattern(GROUP, "messageA", LITERAL)
val notificationMessage2 = Resource(Group, "messageB", LITERAL) val notificationMessage2 = new ResourcePattern(GROUP, "messageB", LITERAL)
notificationListener = new ZkNodeChangeNotificationListener(zkClient, LiteralAclChangeStore.aclChangePath, notificationListener = new ZkNodeChangeNotificationListener(zkClient, LiteralAclChangeStore.aclChangePath,
ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs) ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs)
@ -71,7 +72,7 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness {
TestUtils.waitUntilTrue(() => notificationHandler.received().size == 2 && notificationHandler.received().last == notificationMessage2, TestUtils.waitUntilTrue(() => notificationHandler.received().size == 2 && notificationHandler.received().last == notificationMessage2,
"Failed to send/process notification message in the timeout period.") "Failed to send/process notification message in the timeout period.")
(3 to 10).foreach(i => zkClient.createAclChangeNotification(Resource(Group, "message" + i, LITERAL))) (3 to 10).foreach(i => zkClient.createAclChangeNotification(new ResourcePattern(GROUP, "message" + i, LITERAL)))
TestUtils.waitUntilTrue(() => notificationHandler.received().size == 10, TestUtils.waitUntilTrue(() => notificationHandler.received().size == 10,
s"Expected 10 invocations of processNotifications, but there were ${notificationHandler.received()}") s"Expected 10 invocations of processNotifications, but there were ${notificationHandler.received()}")
@ -84,16 +85,16 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness {
ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs) ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs)
notificationListener.init() notificationListener.init()
zkClient.createAclChangeNotification(Resource(Group, "messageA", LITERAL)) zkClient.createAclChangeNotification(new ResourcePattern(GROUP, "messageA", LITERAL))
zkClient.createAclChangeNotification(Resource(Group, "messageB", LITERAL)) zkClient.createAclChangeNotification(new ResourcePattern(GROUP, "messageB", LITERAL))
zkClient.createAclChangeNotification(Resource(Group, "messageC", LITERAL)) zkClient.createAclChangeNotification(new ResourcePattern(GROUP, "messageC", LITERAL))
TestUtils.waitUntilTrue(() => notificationHandler.received().size == 3, TestUtils.waitUntilTrue(() => notificationHandler.received().size == 3,
s"Expected 2 invocations of processNotifications, but there were ${notificationHandler.received()}") s"Expected 2 invocations of processNotifications, but there were ${notificationHandler.received()}")
} }
private class TestNotificationHandler extends NotificationHandler { private class TestNotificationHandler extends NotificationHandler {
private val messages = ArrayBuffer.empty[Resource] private val messages = ArrayBuffer.empty[ResourcePattern]
@volatile private var throwSize = Option.empty[Int] @volatile private var throwSize = Option.empty[Int]
override def processNotification(notificationMessage: Array[Byte]): Unit = { override def processNotification(notificationMessage: Array[Byte]): Unit = {
@ -103,7 +104,7 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness {
throw new RuntimeException("Oh no, my processing failed!") throw new RuntimeException("Oh no, my processing failed!")
} }
def received(): Seq[Resource] = messages def received(): Seq[ResourcePattern] = messages
def setThrowSize(index: Int): Unit = throwSize = Option(index) def setThrowSize(index: Int): Unit = throwSize = Option(index)
} }

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.acl.AclOperation
import org.junit.Assert.assertEquals import org.junit.Assert.assertEquals
import org.junit.Test import org.junit.Test
@deprecated("Scala Authorizer API classes gave been deprecated", "Since 2.5")
class OperationTest { class OperationTest {
/** /**
* Test round trip conversions between org.apache.kafka.common.acl.AclOperation and * Test round trip conversions between org.apache.kafka.common.acl.AclOperation and

View File

@ -22,6 +22,7 @@ import org.junit.Assert.assertEquals
import org.junit.Test import org.junit.Test
import org.scalatest.Assertions.fail import org.scalatest.Assertions.fail
@deprecated("Scala Authorizer API classes gave been deprecated", "Since 2.5")
class PermissionTypeTest { class PermissionTypeTest {
@Test @Test

View File

@ -22,6 +22,7 @@ import org.junit.Test
import org.scalatest.Assertions.fail import org.scalatest.Assertions.fail
import org.apache.kafka.common.resource.{ResourceType => JResourceType} import org.apache.kafka.common.resource.{ResourceType => JResourceType}
@deprecated("Scala Authorizer API classes gave been deprecated", "Since 2.5")
class ResourceTypeTest { class ResourceTypeTest {
@Test @Test

View File

@ -313,12 +313,12 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
//test remove all acls for resource //test remove all acls for resource
simpleAclAuthorizer.removeAcls(resource) simpleAclAuthorizer.removeAcls(resource)
TestUtils.waitAndVerifyAcls(Set.empty[Acl], simpleAclAuthorizer, resource) TestUtils.waitAndVerifyAcls(Set.empty[Acl], simpleAclAuthorizer, resource)
assertTrue(!zkClient.resourceExists(resource)) assertTrue(!zkClient.resourceExists(resource.toPattern))
//test removing last acl also deletes ZooKeeper path //test removing last acl also deletes ZooKeeper path
acls = changeAclAndVerify(Set.empty[Acl], Set(acl1), Set.empty[Acl]) acls = changeAclAndVerify(Set.empty[Acl], Set(acl1), Set.empty[Acl])
changeAclAndVerify(acls, Set.empty[Acl], acls) changeAclAndVerify(acls, Set.empty[Acl], acls)
assertTrue(!zkClient.resourceExists(resource)) assertTrue(!zkClient.resourceExists(resource.toPattern))
} }
@Test @Test
@ -639,7 +639,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
def testWritesExtendedAclChangeEventIfInterBrokerProtocolNotSet(): Unit = { def testWritesExtendedAclChangeEventIfInterBrokerProtocolNotSet(): Unit = {
givenAuthorizerWithProtocolVersion(Option.empty) givenAuthorizerWithProtocolVersion(Option.empty)
val resource = Resource(Topic, "z_other", PREFIXED) val resource = Resource(Topic, "z_other", PREFIXED)
val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource).bytes, UTF_8) val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource.toPattern).bytes, UTF_8)
simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource)
@ -652,7 +652,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
def testWritesExtendedAclChangeEventWhenInterBrokerProtocolAtLeastKafkaV2(): Unit = { def testWritesExtendedAclChangeEventWhenInterBrokerProtocolAtLeastKafkaV2(): Unit = {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
val resource = Resource(Topic, "z_other", PREFIXED) val resource = Resource(Topic, "z_other", PREFIXED)
val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource).bytes, UTF_8) val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource.toPattern).bytes, UTF_8)
simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource)
@ -665,7 +665,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
def testWritesLiteralWritesLiteralAclChangeEventWhenInterBrokerProtocolLessThanKafkaV2eralAclChangesForOlderProtocolVersions(): Unit = { def testWritesLiteralWritesLiteralAclChangeEventWhenInterBrokerProtocolLessThanKafkaV2eralAclChangesForOlderProtocolVersions(): Unit = {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0))
val resource = Resource(Topic, "z_other", LITERAL) val resource = Resource(Topic, "z_other", LITERAL)
val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource).bytes, UTF_8) val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource.toPattern).bytes, UTF_8)
simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource)
@ -678,7 +678,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
def testWritesLiteralAclChangeEventWhenInterBrokerProtocolIsKafkaV2(): Unit = { def testWritesLiteralAclChangeEventWhenInterBrokerProtocolIsKafkaV2(): Unit = {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
val resource = Resource(Topic, "z_other", LITERAL) val resource = Resource(Topic, "z_other", LITERAL)
val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource).bytes, UTF_8) val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource.toPattern).bytes, UTF_8)
simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource)

View File

@ -22,8 +22,7 @@ import java.util.UUID
import java.util.concurrent.{Executors, Semaphore, TimeUnit} import java.util.concurrent.{Executors, Semaphore, TimeUnit}
import kafka.api.{ApiVersion, KAFKA_2_0_IV0, KAFKA_2_0_IV1} import kafka.api.{ApiVersion, KAFKA_2_0_IV0, KAFKA_2_0_IV1}
import kafka.security.auth.Resource import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString}
import kafka.security.authorizer.AuthorizerUtils.{WildcardHost, WildcardPrincipal}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.zk.{ZkAclStore, ZooKeeperTestHarness} import kafka.zk.{ZkAclStore, ZooKeeperTestHarness}
@ -37,6 +36,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.{RequestContext, RequestHeader} import org.apache.kafka.common.requests.{RequestContext, RequestHeader}
import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType}
import org.apache.kafka.common.resource.Resource.CLUSTER_NAME
import org.apache.kafka.common.resource.ResourcePattern.WILDCARD_RESOURCE import org.apache.kafka.common.resource.ResourcePattern.WILDCARD_RESOURCE
import org.apache.kafka.common.resource.ResourceType._ import org.apache.kafka.common.resource.ResourceType._
import org.apache.kafka.common.resource.PatternType.{LITERAL, MATCH, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, MATCH, PREFIXED}
@ -52,14 +52,14 @@ import scala.compat.java8.OptionConverters._
class AclAuthorizerTest extends ZooKeeperTestHarness { class AclAuthorizerTest extends ZooKeeperTestHarness {
private val allowReadAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, READ, ALLOW) private val allowReadAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, READ, ALLOW)
private val allowWriteAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, WRITE, ALLOW) private val allowWriteAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, WRITE, ALLOW)
private val denyReadAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, READ, DENY) private val denyReadAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, READ, DENY)
private val wildCardResource = new ResourcePattern(TOPIC, WILDCARD_RESOURCE, LITERAL) private val wildCardResource = new ResourcePattern(TOPIC, WILDCARD_RESOURCE, LITERAL)
private val prefixedResource = new ResourcePattern(TOPIC, "foo", PREFIXED) private val prefixedResource = new ResourcePattern(TOPIC, "foo", PREFIXED)
private val clusterResource = new ResourcePattern(CLUSTER, Resource.ClusterResourceName, LITERAL) private val clusterResource = new ResourcePattern(CLUSTER, CLUSTER_NAME, LITERAL)
private val wildcardPrincipal = JSecurityUtils.parseKafkaPrincipal(WildcardPrincipal) private val wildcardPrincipal = JSecurityUtils.parseKafkaPrincipal(WildcardPrincipalString)
private val aclAuthorizer = new AclAuthorizer private val aclAuthorizer = new AclAuthorizer
private val aclAuthorizer2 = new AclAuthorizer private val aclAuthorizer2 = new AclAuthorizer
@ -205,7 +205,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
val host = InetAddress.getByName("192.168.2.1") val host = InetAddress.getByName("192.168.2.1")
val session = newRequestContext(user, host) val session = newRequestContext(user, host)
val allowAll = new AccessControlEntry(WildcardPrincipal, WildcardHost, AclOperation.ALL, ALLOW) val allowAll = new AccessControlEntry(WildcardPrincipalString, WildcardHost, AclOperation.ALL, ALLOW)
val denyAcl = new AccessControlEntry(user.toString, host.getHostAddress, AclOperation.ALL, DENY) val denyAcl = new AccessControlEntry(user.toString, host.getHostAddress, AclOperation.ALL, DENY)
val acls = Set(allowAll, denyAcl) val acls = Set(allowAll, denyAcl)
@ -216,7 +216,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
@Test @Test
def testAllowAllAccess(): Unit = { def testAllowAllAccess(): Unit = {
val allowAllAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, AclOperation.ALL, ALLOW) val allowAllAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, AclOperation.ALL, ALLOW)
changeAclAndVerify(Set.empty, Set(allowAllAcl), Set.empty) changeAclAndVerify(Set.empty, Set(allowAllAcl), Set.empty)
@ -226,7 +226,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
@Test @Test
def testSuperUserHasAccess(): Unit = { def testSuperUserHasAccess(): Unit = {
val denyAllAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, AclOperation.ALL, DENY) val denyAllAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, AclOperation.ALL, DENY)
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty) changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
@ -242,7 +242,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
*/ */
@Test @Test
def testSuperUserWithCustomPrincipalHasAccess(): Unit = { def testSuperUserWithCustomPrincipalHasAccess(): Unit = {
val denyAllAcl = new AccessControlEntry(WildcardPrincipal, WildcardHost, AclOperation.ALL, DENY) val denyAllAcl = new AccessControlEntry(WildcardPrincipalString, WildcardHost, AclOperation.ALL, DENY)
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty) changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
val session = newRequestContext(new CustomPrincipal(KafkaPrincipal.USER_TYPE, "superuser1"), InetAddress.getByName("192.0.4.4")) val session = newRequestContext(new CustomPrincipal(KafkaPrincipal.USER_TYPE, "superuser1"), InetAddress.getByName("192.0.4.4"))
@ -338,12 +338,12 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
//test remove all acls for resource //test remove all acls for resource
removeAcls(aclAuthorizer, Set.empty, resource) removeAcls(aclAuthorizer, Set.empty, resource)
TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], aclAuthorizer, resource) TestUtils.waitAndVerifyAcls(Set.empty[AccessControlEntry], aclAuthorizer, resource)
assertTrue(!zkClient.resourceExists(AuthorizerUtils.convertToResource(resource))) assertTrue(!zkClient.resourceExists(resource))
//test removing last acl also deletes ZooKeeper path //test removing last acl also deletes ZooKeeper path
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)
assertTrue(!zkClient.resourceExists(AuthorizerUtils.convertToResource(resource))) assertTrue(!zkClient.resourceExists(resource))
} }
@Test @Test
@ -678,7 +678,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
1, getAcls(aclAuthorizer, new KafkaPrincipal(principal.getPrincipalType, principal.getName)).size) 1, getAcls(aclAuthorizer, new KafkaPrincipal(principal.getPrincipalType, principal.getName)).size)
removeAcls(aclAuthorizer, Set.empty, resource) removeAcls(aclAuthorizer, Set.empty, resource)
val aclOnWildcardPrincipal = new AccessControlEntry(WildcardPrincipal, WildcardHost, WRITE, ALLOW) val aclOnWildcardPrincipal = new AccessControlEntry(WildcardPrincipalString, WildcardHost, WRITE, ALLOW)
addAcls(aclAuthorizer, Set(aclOnWildcardPrincipal), resource) addAcls(aclAuthorizer, Set(aclOnWildcardPrincipal), resource)
assertEquals("acl on wildcard should be returned for wildcard request", assertEquals("acl on wildcard should be returned for wildcard request",
@ -734,7 +734,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
givenAuthorizerWithProtocolVersion(Option.empty) givenAuthorizerWithProtocolVersion(Option.empty)
val resource = new ResourcePattern(TOPIC, "z_other", PREFIXED) val resource = new ResourcePattern(TOPIC, "z_other", PREFIXED)
val expected = new String(ZkAclStore(PREFIXED).changeStore val expected = new String(ZkAclStore(PREFIXED).changeStore
.createChangeNode(AuthorizerUtils.convertToResource(resource)).bytes, UTF_8) .createChangeNode(resource).bytes, UTF_8)
addAcls(aclAuthorizer, Set(denyReadAcl), resource) addAcls(aclAuthorizer, Set(denyReadAcl), resource)
@ -748,7 +748,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
val resource = new ResourcePattern(TOPIC, "z_other", PREFIXED) val resource = new ResourcePattern(TOPIC, "z_other", PREFIXED)
val expected = new String(ZkAclStore(PREFIXED).changeStore val expected = new String(ZkAclStore(PREFIXED).changeStore
.createChangeNode(AuthorizerUtils.convertToResource(resource)).bytes, UTF_8) .createChangeNode(resource).bytes, UTF_8)
addAcls(aclAuthorizer, Set(denyReadAcl), resource) addAcls(aclAuthorizer, Set(denyReadAcl), resource)
@ -762,7 +762,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0))
val resource = new ResourcePattern(TOPIC, "z_other", LITERAL) val resource = new ResourcePattern(TOPIC, "z_other", LITERAL)
val expected = new String(ZkAclStore(LITERAL).changeStore val expected = new String(ZkAclStore(LITERAL).changeStore
.createChangeNode(AuthorizerUtils.convertToResource(resource)).bytes, UTF_8) .createChangeNode(resource).bytes, UTF_8)
addAcls(aclAuthorizer, Set(denyReadAcl), resource) addAcls(aclAuthorizer, Set(denyReadAcl), resource)
@ -776,7 +776,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
val resource = new ResourcePattern(TOPIC, "z_other", LITERAL) val resource = new ResourcePattern(TOPIC, "z_other", LITERAL)
val expected = new String(ZkAclStore(LITERAL).changeStore val expected = new String(ZkAclStore(LITERAL).changeStore
.createChangeNode(AuthorizerUtils.convertToResource(resource)).bytes, UTF_8) .createChangeNode(resource).bytes, UTF_8)
addAcls(aclAuthorizer, Set(denyReadAcl), resource) addAcls(aclAuthorizer, Set(denyReadAcl), resource)

View File

@ -14,17 +14,20 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package kafka.security.auth package kafka.security.authorizer
import java.nio.charset.StandardCharsets.UTF_8 import java.nio.charset.StandardCharsets.UTF_8
import kafka.utils.Json import kafka.utils.Json
import org.apache.kafka.common.acl.AclOperation.READ
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.junit.{Assert, Test} import org.junit.{Assert, Test}
import org.scalatestplus.junit.JUnitSuite import org.scalatestplus.junit.JUnitSuite
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
class AclTest extends JUnitSuite { class AclEntryTest extends JUnitSuite {
val AclJson = "{\"version\": 1, \"acls\": [{\"host\": \"host1\",\"permissionType\": \"Deny\",\"operation\": \"READ\", \"principal\": \"User:alice\" }, " + val AclJson = "{\"version\": 1, \"acls\": [{\"host\": \"host1\",\"permissionType\": \"Deny\",\"operation\": \"READ\", \"principal\": \"User:alice\" }, " +
"{ \"host\": \"*\" , \"permissionType\": \"Allow\", \"operation\": \"Read\", \"principal\": \"User:bob\" }, " + "{ \"host\": \"*\" , \"permissionType\": \"Allow\", \"operation\": \"Read\", \"principal\": \"User:bob\" }, " +
@ -32,15 +35,15 @@ class AclTest extends JUnitSuite {
@Test @Test
def testAclJsonConversion(): Unit = { def testAclJsonConversion(): Unit = {
val acl1 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), Deny, "host1" , Read) val acl1 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), DENY, "host1" , READ)
val acl2 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Allow, "*", Read) val acl2 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), ALLOW, "*", READ)
val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Deny, "host1", Read) val acl3 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), DENY, "host1", READ)
val acls = Set[Acl](acl1, acl2, acl3) val acls = Set[AclEntry](acl1, acl2, acl3)
val jsonAcls = Json.encodeAsBytes(Acl.toJsonCompatibleMap(acls).asJava) val jsonAcls = Json.encodeAsBytes(AclEntry.toJsonCompatibleMap(acls).asJava)
Assert.assertEquals(acls, Acl.fromBytes(jsonAcls)) Assert.assertEquals(acls, AclEntry.fromBytes(jsonAcls))
Assert.assertEquals(acls, Acl.fromBytes(AclJson.getBytes(UTF_8))) Assert.assertEquals(acls, AclEntry.fromBytes(AclJson.getBytes(UTF_8)))
} }
} }

View File

@ -22,8 +22,8 @@ import java.nio.ByteBuffer
import java.util.{Base64, Properties} import java.util.{Base64, Properties}
import kafka.network.RequestChannel.Session import kafka.network.RequestChannel.Session
import kafka.security.auth.Acl.WildCardHost
import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils} import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils}
import kafka.security.authorizer.AclEntry.WildcardHost
import kafka.server.{CreateTokenResult, Defaults, DelegationTokenManager, KafkaConfig} import kafka.server.{CreateTokenResult, Defaults, DelegationTokenManager, KafkaConfig}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.zk.{KafkaZkClient, ZooKeeperTestHarness} import kafka.zk.{KafkaZkClient, ZooKeeperTestHarness}
@ -287,7 +287,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness {
//get all tokens for multiple owners (owner1, renewer4) and with permission //get all tokens for multiple owners (owner1, renewer4) and with permission
createAcl(new AclBinding(new ResourcePattern(DELEGATION_TOKEN, tokenId3, LITERAL), createAcl(new AclBinding(new ResourcePattern(DELEGATION_TOKEN, tokenId3, LITERAL),
new AccessControlEntry(owner1.toString, WildCardHost, DESCRIBE, ALLOW))) new AccessControlEntry(owner1.toString, WildcardHost, DESCRIBE, ALLOW)))
tokens = getTokens(tokenManager, aclAuthorizer, hostSession, owner1, List(owner1, renewer4)) tokens = getTokens(tokenManager, aclAuthorizer, hostSession, owner1, List(owner1, renewer4))
assert(tokens.size == 3) assert(tokens.size == 3)
@ -302,7 +302,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness {
//get all tokens for multiple owners (renewer2, renewer3) which are token renewers principals and with permissions //get all tokens for multiple owners (renewer2, renewer3) which are token renewers principals and with permissions
hostSession = Session(renewer2, InetAddress.getByName("192.168.1.1")) hostSession = Session(renewer2, InetAddress.getByName("192.168.1.1"))
createAcl(new AclBinding(new ResourcePattern(DELEGATION_TOKEN, tokenId2, LITERAL), createAcl(new AclBinding(new ResourcePattern(DELEGATION_TOKEN, tokenId2, LITERAL),
new AccessControlEntry(renewer2.toString, WildCardHost, DESCRIBE, ALLOW))) new AccessControlEntry(renewer2.toString, WildcardHost, DESCRIBE, ALLOW)))
tokens = getTokens(tokenManager, aclAuthorizer, hostSession, renewer2, List(renewer2, renewer3)) tokens = getTokens(tokenManager, aclAuthorizer, hostSession, renewer2, List(renewer2, renewer3))
assert(tokens.size == 2) assert(tokens.size == 2)

View File

@ -33,7 +33,7 @@ import javax.net.ssl.X509TrustManager
import kafka.api._ import kafka.api._
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.log._ import kafka.log._
import kafka.security.auth.{Acl, Authorizer, Resource} import kafka.security.auth.{Acl, Authorizer => LegacyAuthorizer, Resource}
import kafka.server._ import kafka.server._
import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.checkpoints.OffsetCheckpointFile
import Implicits._ import Implicits._
@ -59,7 +59,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, IntegerSerializer, Serializer} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, IntegerSerializer, Serializer}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.common.utils.Utils._
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
import org.apache.zookeeper.KeeperException.SessionExpiredException import org.apache.zookeeper.KeeperException.SessionExpiredException
import org.apache.zookeeper.ZooDefs._ import org.apache.zookeeper.ZooDefs._
@ -1192,7 +1192,7 @@ object TestUtils extends Logging {
trustManager trustManager
} }
def waitAndVerifyAcls(expected: Set[AccessControlEntry], authorizer: JAuthorizer, resource: ResourcePattern) = { def waitAndVerifyAcls(expected: Set[AccessControlEntry], authorizer: Authorizer, resource: ResourcePattern) = {
val newLine = scala.util.Properties.lineSeparator val newLine = scala.util.Properties.lineSeparator
val filter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY) val filter = new AclBindingFilter(resource.toFilter, AccessControlEntryFilter.ANY)
@ -1201,7 +1201,8 @@ object TestUtils extends Logging {
s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", waitTimeMs = JTestUtils.DEFAULT_MAX_WAIT_MS) s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", waitTimeMs = JTestUtils.DEFAULT_MAX_WAIT_MS)
} }
def waitAndVerifyAcls(expected: Set[Acl], authorizer: Authorizer, resource: Resource) = { @deprecated("Use org.apache.kafka.server.authorizer.Authorizer", "Since 2.5")
def waitAndVerifyAcls(expected: Set[Acl], authorizer: LegacyAuthorizer, resource: Resource) = {
val newLine = scala.util.Properties.lineSeparator val newLine = scala.util.Properties.lineSeparator
waitUntilTrue(() => authorizer.getAcls(resource) == expected, waitUntilTrue(() => authorizer.getAcls(resource) == expected,

View File

@ -23,7 +23,6 @@ import java.util.concurrent.{CountDownLatch, TimeUnit}
import kafka.api.{ApiVersion, LeaderAndIsr} import kafka.api.{ApiVersion, LeaderAndIsr}
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.security.auth._
import kafka.server.ConfigType import kafka.server.ConfigType
import kafka.utils.CoreUtils import kafka.utils.CoreUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
@ -41,9 +40,14 @@ import scala.collection.mutable.ArrayBuffer
import scala.collection.{Seq, mutable} import scala.collection.{Seq, mutable}
import scala.util.Random import scala.util.Random
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.security.authorizer.AclEntry
import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
import kafka.zookeeper._ import kafka.zookeeper._
import org.apache.kafka.common.acl.AclOperation.READ
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC}
import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.JaasUtils
import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.ZooDefs
import org.apache.zookeeper.data.Stat import org.apache.zookeeper.data.Stat
@ -465,7 +469,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
ZkAclStore.stores.foreach(store => { ZkAclStore.stores.foreach(store => {
assertFalse(zkClient.pathExists(store.aclPath)) assertFalse(zkClient.pathExists(store.aclPath))
assertFalse(zkClient.pathExists(store.changeStore.aclChangePath)) assertFalse(zkClient.pathExists(store.changeStore.aclChangePath))
ResourceType.values.foreach(resource => assertFalse(zkClient.pathExists(store.path(resource)))) AclEntry.ResourceTypes.foreach(resource => assertFalse(zkClient.pathExists(store.path(resource))))
}) })
// create acl paths // create acl paths
@ -474,10 +478,10 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
ZkAclStore.stores.foreach(store => { ZkAclStore.stores.foreach(store => {
assertTrue(zkClient.pathExists(store.aclPath)) assertTrue(zkClient.pathExists(store.aclPath))
assertTrue(zkClient.pathExists(store.changeStore.aclChangePath)) assertTrue(zkClient.pathExists(store.changeStore.aclChangePath))
ResourceType.values.foreach(resource => assertTrue(zkClient.pathExists(store.path(resource)))) AclEntry.ResourceTypes.foreach(resource => assertTrue(zkClient.pathExists(store.path(resource))))
val resource1 = new Resource(Topic, UUID.randomUUID().toString, store.patternType) val resource1 = new ResourcePattern(TOPIC, UUID.randomUUID().toString, store.patternType)
val resource2 = new Resource(Topic, UUID.randomUUID().toString, store.patternType) val resource2 = new ResourcePattern(TOPIC, UUID.randomUUID().toString, store.patternType)
// try getting acls for non-existing resource // try getting acls for non-existing resource
var versionedAcls = zkClient.getVersionedAclsForResource(resource1) var versionedAcls = zkClient.getVersionedAclsForResource(resource1)
@ -486,9 +490,9 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
assertFalse(zkClient.resourceExists(resource1)) assertFalse(zkClient.resourceExists(resource1))
val acl1 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), Deny, "host1" , Read) val acl1 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), DENY, "host1" , READ)
val acl2 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Allow, "*", Read) val acl2 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), ALLOW, "*", READ)
val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Deny, "host1", Read) val acl3 = AclEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), DENY, "host1", READ)
// Conditional set should fail if path not created // Conditional set should fail if path not created
assertFalse(zkClient.conditionalSetAclsForResource(resource1, Set(acl1, acl3), 0)._1) assertFalse(zkClient.conditionalSetAclsForResource(resource1, Set(acl1, acl3), 0)._1)
@ -513,10 +517,10 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
assertEquals(1, versionedAcls.zkVersion) assertEquals(1, versionedAcls.zkVersion)
//get resource Types //get resource Types
assertTrue(ResourceType.values.map( rt => rt.name).toSet == zkClient.getResourceTypes(store.patternType).toSet) assertEquals(AclEntry.ResourceTypes.map(SecurityUtils.resourceTypeName), zkClient.getResourceTypes(store.patternType).toSet)
//get resource name //get resource name
val resourceNames = zkClient.getResourceNames(store.patternType, Topic) val resourceNames = zkClient.getResourceNames(store.patternType, TOPIC)
assertEquals(2, resourceNames.size) assertEquals(2, resourceNames.size)
assertTrue(Set(resource1.name,resource2.name) == resourceNames.toSet) assertTrue(Set(resource1.name,resource2.name) == resourceNames.toSet)
@ -529,8 +533,8 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
//delete with valid expected zk version //delete with valid expected zk version
assertTrue(zkClient.conditionalDelete(resource2, 0)) assertTrue(zkClient.conditionalDelete(resource2, 0))
zkClient.createAclChangeNotification(Resource(Group, "resource1", store.patternType)) zkClient.createAclChangeNotification(new ResourcePattern(GROUP, "resource1", store.patternType))
zkClient.createAclChangeNotification(Resource(Topic, "resource2", store.patternType)) zkClient.createAclChangeNotification(new ResourcePattern(TOPIC, "resource2", store.patternType))
assertEquals(2, zkClient.getChildren(store.changeStore.aclChangePath).size) assertEquals(2, zkClient.getChildren(store.changeStore.aclChangePath).size)

View File

@ -33,6 +33,11 @@
<li>Scala 2.11 is no longer supported. See <li>Scala 2.11 is no longer supported. See
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-531%3A+Drop+support+for+Scala+2.11+in+Kafka+2.5">KIP-531</a> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-531%3A+Drop+support+for+Scala+2.11+in+Kafka+2.5">KIP-531</a>
for details.</li> for details.</li>
<li>All Scala classes from the package <code>kafka.security.auth</code> have been deprecated. See
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface">KIP-504</a>
for details of the new Java authorizer API added in 2.4.0. Note that <code>kafka.security.auth.Authorizer</code>
and <code>kafka.security.auth.SimpleAclAuthorizer</code> were deprecated in 2.4.0.
</li>
</ul> </ul>
<h4><a id="upgrade_2_4_0" href="#upgrade_2_4_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x or 2.3.x to 2.4.0</a></h4> <h4><a id="upgrade_2_4_0" href="#upgrade_2_4_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x or 2.3.x to 2.4.0</a></h4>
@ -156,6 +161,13 @@
<code>onPartitionsAssigned</code> callback will however always be called, even with an empty set of partitions, as a way to notify users of a rebalance event (this is true for both cooperative and eager). For details on <code>onPartitionsAssigned</code> callback will however always be called, even with an empty set of partitions, as a way to notify users of a rebalance event (this is true for both cooperative and eager). For details on
the new callback semantics, see the <a href="https://kafka.apache.org/24/javadoc/index.html?org/apache/kafka/clients/consumer/ConsumerRebalanceListener.html">ConsumerRebalanceListener javadocs</a>. the new callback semantics, see the <a href="https://kafka.apache.org/24/javadoc/index.html?org/apache/kafka/clients/consumer/ConsumerRebalanceListener.html">ConsumerRebalanceListener javadocs</a>.
</li> </li>
<li>The Scala trait <code>kafka.security.auth.Authorizer</code> has been deprecated and replaced with a new Java API
<code>org.apache.kafka.server.authorizer.Authorizer</code>. The authorizer implementation class
<code>kafka.security.auth.SimpleAclAuthorizer</code> has also been deprecated and replaced with a new
implementation <code>kafka.security.authorizer.AclAuthorizer</code>. <code>AclAuthorizer</code> uses features
supported by the new API to improve authorization logging and is compatible with <code>SimpleAclAuthorizer</code>.
For more details, see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface">KIP-504</a>.
</li>
</ul> </ul>
<h4><a id="upgrade_2_3_0" href="#upgrade_2_3_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0</a></h4> <h4><a id="upgrade_2_3_0" href="#upgrade_2_3_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0</a></h4>