You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by cm...@apache.org on 2022/10/24 21:44:42 UTC

[kafka] branch 3.3 updated: MINOR: Update security docs for kraft Authorizer configuration (#12673)

This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
     new 23b04ec1216 MINOR: Update security docs for kraft Authorizer configuration (#12673)
23b04ec1216 is described below

commit 23b04ec12160456b4b67997ae8372894d9c1359d
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Wed Sep 21 19:38:59 2022 -0700

    MINOR: Update security docs for kraft Authorizer configuration (#12673)
    
    Update security documentation to describe how to configure the KRaft `Authorizer` implementation and include a note about principal forwarding.
    
    Additionally, this patch renames `KafkaConfig.Defaults.DefaultPrincipalSerde` to `DefaultPrincipalBuilder` since the former is somewhat misleading.
    
    Reviewers: David Arthur <mu...@gmail.com>
---
 core/src/main/scala/kafka/server/KafkaConfig.scala |  4 ++--
 docs/security.html                                 | 23 +++++++++++++++++++++-
 2 files changed, 24 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 497a904c2c5..2df40e97b72 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -259,7 +259,7 @@ object Defaults {
     /** ********* General Security configuration ***********/
   val ConnectionsMaxReauthMsDefault = 0L
   val DefaultServerMaxMaxReceiveSize = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE
-  val DefaultPrincipalSerde = classOf[DefaultKafkaPrincipalBuilder]
+  val DefaultPrincipalBuilder = classOf[DefaultKafkaPrincipalBuilder]
 
   /** ********* Sasl configuration ***********/
   val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM
@@ -1323,7 +1323,7 @@ object KafkaConfig {
       .define(securityProviderClassProp, STRING, null, LOW, securityProviderClassDoc)
 
       /** ********* SSL Configuration ****************/
-      .define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalSerde, MEDIUM, PrincipalBuilderClassDoc)
+      .define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalBuilder, MEDIUM, PrincipalBuilderClassDoc)
       .define(SslProtocolProp, STRING, Defaults.SslProtocol, MEDIUM, SslProtocolDoc)
       .define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc)
       .define(SslEnabledProtocolsProp, LIST, Defaults.SslEnabledProtocols, MEDIUM, SslEnabledProtocolsDoc)
diff --git a/docs/security.html b/docs/security.html
index 92090e2d427..6bf15f72462 100644
--- a/docs/security.html
+++ b/docs/security.html
@@ -1269,11 +1269,32 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
 
     For Zookeeper-based clusters, the provided implementation is configured as follows:
     <pre class="line-numbers"><code class="language-text">authorizer.class.name=kafka.security.authorizer.AclAuthorizer</code></pre>
-    Kafka acls are defined in the general format of "Principal P is [Allowed/Denied] Operation O From Host H on any Resource R matching ResourcePattern RP". You can read more about the acl structure in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface">KIP-11</a> and resource patterns in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs">KIP-290</a>. In order to add, remove or list acls you can use th [...]
+    For KRaft clusters, use the following configuration on all nodes (brokers, controllers, or combined broker/controller nodes):
+    <pre class="line-numbers"><code class="language-text">authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer</code></pre>
+
+    Kafka ACLs are defined in the general format of "Principal {P} is [Allowed|Denied] Operation {O} From Host {H} on any Resource {R} matching ResourcePattern {RP}".
+    You can read more about the ACL structure in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface">KIP-11</a> and
+    resource patterns in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs">KIP-290</a>.
+    In order to add, remove, or list ACLs, you can use the Kafka ACL CLI <code>kafka-acls.sh</code>. By default, if no ResourcePatterns match a specific Resource R,
+    then R has no associated ACLs, and therefore no one other than super users is allowed to access R.
+    If you want to change that behavior, you can include the following in server.properties.
     <pre class="line-numbers"><code class="language-text">allow.everyone.if.no.acl.found=true</code></pre>
     One can also add super users in server.properties like the following (note that the delimiter is semicolon since SSL user names may contain comma). Default PrincipalType string "User" is case sensitive.
     <pre class="line-numbers"><code class="language-text">super.users=User:Bob;User:Alice</code></pre>
 
+    <h5 class="anchor-heading"><a id="kraft_principal_forwarding" class="anchor-link"></a><a href="#kraft_principal_forwarding">KRaft Principal Forwarding</a></h5>
+
+    In KRaft clusters, admin requests such as <code>CreateTopics</code> and <code>DeleteTopics</code> are sent to the broker listeners by the client. The broker then forwards the request to the active controller through the first listener configured in <code>controller.listener.names</code>.
+    Authorization of these requests is done on the controller node. This is achieved by way of an <code>Envelope</code> request which packages both the underlying request from the client as well as the client principal.
+    When the controller receives the forwarded <code>Envelope</code> request from the broker, it first authorizes the <code>Envelope</code> request using the authenticated broker principal.
+    Then it authorizes the underlying request using the forwarded principal.
+
+    <br>All of this implies that Kafka must understand how to serialize and deserialize the client principal. The authentication framework allows for customized principals by overriding the <code>principal.builder.class</code> configuration.
+    In order for customized principals to work with KRaft, the configured class must implement <code>org.apache.kafka.common.security.auth.KafkaPrincipalSerde</code> so that Kafka knows how to serialize and deserialize the principals.
+    The default implementation <code>org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder</code> uses the Kafka RPC format defined in the source code: <code>clients/src/main/resources/common/message/DefaultPrincipalData.json</code>.
+
+    For more detail about request forwarding in KRaft, see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller">KIP-590</a>
+    
     <h5 class="anchor-heading"><a id="security_authz_ssl" class="anchor-link"></a><a href="#security_authz_ssl">Customizing SSL User Name</a></h5>
 
     By default, the SSL user name will be of the form "CN=writeuser,OU=Unknown,O=Unknown,L=Unknown,ST=Unknown,C=Unknown". One can change that by setting <code>ssl.principal.mapping.rules</code> to a customized rule in server.properties.