mirror of https://github.com/apache/kafka.git
MINOR: Delete KafkaSecurityConfigs class (#16113)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
cc269b0d43
commit
3f3f3ac155
|
@ -17,6 +17,9 @@
|
|||
package org.apache.kafka.common.config.internals;
|
||||
|
||||
import org.apache.kafka.common.config.SaslConfigs;
|
||||
import org.apache.kafka.common.config.SslClientAuth;
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder;
|
||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -30,20 +33,28 @@ import java.util.List;
|
|||
public class BrokerSecurityConfigs {
|
||||
|
||||
public static final String PRINCIPAL_BUILDER_CLASS_CONFIG = "principal.builder.class";
|
||||
public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = "sasl.kerberos.principal.to.local.rules";
|
||||
public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth";
|
||||
public static final String SASL_ENABLED_MECHANISMS_CONFIG = "sasl.enabled.mechanisms";
|
||||
public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS = "sasl.server.callback.handler.class";
|
||||
public static final String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = "ssl.principal.mapping.rules";
|
||||
public static final String CONNECTIONS_MAX_REAUTH_MS = "connections.max.reauth.ms";
|
||||
public static final long DEFAULT_CONNECTIONS_MAX_REAUTH_MS = 0L;
|
||||
public static final int DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE = 524288;
|
||||
public static final String SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG = "sasl.server.max.receive.size";
|
||||
public static final String SSL_ALLOW_DN_CHANGES_CONFIG = "ssl.allow.dn.changes";
|
||||
public static final boolean DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE = false;
|
||||
public static final String SSL_ALLOW_SAN_CHANGES_CONFIG = "ssl.allow.san.changes";
|
||||
public static final boolean DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE = false;
|
||||
|
||||
public static final String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = "ssl.principal.mapping.rules";
|
||||
public static final String DEFAULT_SSL_PRINCIPAL_MAPPING_RULES = "DEFAULT";
|
||||
public static final String SSL_PRINCIPAL_MAPPING_RULES_DOC = "A list of rules for mapping from distinguished name" +
|
||||
" from the client certificate to short name. The rules are evaluated in order and the first rule that matches" +
|
||||
" a principal name is used to map it to a short name. Any later rules in the list are ignored. By default," +
|
||||
" distinguished name of the X.500 certificate will be the principal. For more details on the format please" +
|
||||
" see <a href=\"#security_authz\"> security authorization and acls</a>. Note that this configuration is ignored" +
|
||||
" if an extension of KafkaPrincipalBuilder is provided by the <code>" + PRINCIPAL_BUILDER_CLASS_CONFIG + "</code>" +
|
||||
" configuration.";
|
||||
|
||||
public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = "sasl.kerberos.principal.to.local.rules";
|
||||
public static final List<String> DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = Collections.singletonList(DEFAULT_SSL_PRINCIPAL_MAPPING_RULES);
|
||||
public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = "A list of rules for mapping from principal " +
|
||||
"names to short names (typically operating system usernames). The rules are evaluated in order and the " +
|
||||
"first rule that matches a principal name is used to map it to a short name. Any later rules in the list are " +
|
||||
"ignored. By default, principal names of the form <code>{username}/{hostname}@{REALM}</code> are mapped " +
|
||||
"to <code>{username}</code>. For more details on the format please see <a href=\"#security_authz\"> " +
|
||||
"security authorization and acls</a>. Note that this configuration is ignored if an extension of " +
|
||||
"<code>KafkaPrincipalBuilder</code> is provided by the <code>" + PRINCIPAL_BUILDER_CLASS_CONFIG + "</code> configuration.";
|
||||
|
||||
public static final Class<? extends KafkaPrincipalBuilder> PRINCIPAL_BUILDER_CLASS_DEFAULT = DefaultKafkaPrincipalBuilder.class;
|
||||
public static final String PRINCIPAL_BUILDER_CLASS_DOC = "The fully qualified name of a class that implements the " +
|
||||
"KafkaPrincipalBuilder interface, which is used to build the KafkaPrincipal object used during " +
|
||||
"authorization. If no principal builder is defined, the default behavior depends " +
|
||||
|
@ -54,24 +65,8 @@ public class BrokerSecurityConfigs {
|
|||
"rules defined by <code>" + SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG + "</code> if GSSAPI is in use, " +
|
||||
"and the SASL authentication ID for other mechanisms. For PLAINTEXT, the principal will be ANONYMOUS.";
|
||||
|
||||
public static final String SSL_PRINCIPAL_MAPPING_RULES_DOC = "A list of rules for mapping from distinguished name" +
|
||||
" from the client certificate to short name. The rules are evaluated in order and the first rule that matches" +
|
||||
" a principal name is used to map it to a short name. Any later rules in the list are ignored. By default," +
|
||||
" distinguished name of the X.500 certificate will be the principal. For more details on the format please" +
|
||||
" see <a href=\"#security_authz\"> security authorization and acls</a>. Note that this configuration is ignored" +
|
||||
" if an extension of KafkaPrincipalBuilder is provided by the <code>" + PRINCIPAL_BUILDER_CLASS_CONFIG + "</code>" +
|
||||
" configuration.";
|
||||
public static final String DEFAULT_SSL_PRINCIPAL_MAPPING_RULES = "DEFAULT";
|
||||
|
||||
public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = "A list of rules for mapping from principal " +
|
||||
"names to short names (typically operating system usernames). The rules are evaluated in order and the " +
|
||||
"first rule that matches a principal name is used to map it to a short name. Any later rules in the list are " +
|
||||
"ignored. By default, principal names of the form <code>{username}/{hostname}@{REALM}</code> are mapped " +
|
||||
"to <code>{username}</code>. For more details on the format please see <a href=\"#security_authz\"> " +
|
||||
"security authorization and acls</a>. Note that this configuration is ignored if an extension of " +
|
||||
"<code>KafkaPrincipalBuilder</code> is provided by the <code>" + PRINCIPAL_BUILDER_CLASS_CONFIG + "</code> configuration.";
|
||||
public static final List<String> DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = Collections.singletonList("DEFAULT");
|
||||
|
||||
public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth";
|
||||
public static final String SSL_CLIENT_AUTH_DEFAULT = SslClientAuth.NONE.toString();
|
||||
public static final String SSL_CLIENT_AUTH_DOC = "Configures kafka broker to request client authentication."
|
||||
+ " The following settings are common: "
|
||||
+ " <ul>"
|
||||
|
@ -81,29 +76,39 @@ public class BrokerSecurityConfigs {
|
|||
+ " <li><code>ssl.client.auth=none</code> This means client authentication is not needed."
|
||||
+ "</ul>";
|
||||
|
||||
public static final String SASL_ENABLED_MECHANISMS_CONFIG = "sasl.enabled.mechanisms";
|
||||
public static final List<String> DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(SaslConfigs.GSSAPI_MECHANISM);
|
||||
public static final String SASL_ENABLED_MECHANISMS_DOC = "The list of SASL mechanisms enabled in the Kafka server. "
|
||||
+ "The list may contain any mechanism for which a security provider is available. "
|
||||
+ "Only GSSAPI is enabled by default.";
|
||||
public static final List<String> DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(SaslConfigs.GSSAPI_MECHANISM);
|
||||
|
||||
public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG = "sasl.server.callback.handler.class";
|
||||
public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC = "The fully qualified name of a SASL server callback handler "
|
||||
+ "class that implements the AuthenticateCallbackHandler interface. Server callback handlers must be prefixed with "
|
||||
+ "listener prefix and SASL mechanism name in lower-case. For example, "
|
||||
+ "listener.name.sasl_ssl.plain.sasl.server.callback.handler.class=com.example.CustomPlainCallbackHandler.";
|
||||
|
||||
public static final String CONNECTIONS_MAX_REAUTH_MS_CONFIG = "connections.max.reauth.ms";
|
||||
public static final long DEFAULT_CONNECTIONS_MAX_REAUTH_MS = 0L;
|
||||
public static final String CONNECTIONS_MAX_REAUTH_MS_DOC = "When explicitly set to a positive number (the default is 0, not a positive number), "
|
||||
+ "a session lifetime that will not exceed the configured value will be communicated to v2.2.0 or later clients when they authenticate. "
|
||||
+ "The broker will disconnect any such connection that is not re-authenticated within the session lifetime and that is then subsequently "
|
||||
+ "used for any purpose other than re-authentication. Configuration names can optionally be prefixed with listener prefix and SASL "
|
||||
+ "mechanism name in lower-case. For example, listener.name.sasl_ssl.oauthbearer.connections.max.reauth.ms=3600000";
|
||||
|
||||
public static final String SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG = "sasl.server.max.receive.size";
|
||||
public static final int DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE = 524288;
|
||||
public static final String SASL_SERVER_MAX_RECEIVE_SIZE_DOC = "The maximum receive size allowed before and during initial SASL authentication." +
|
||||
" Default receive size is 512KB. GSSAPI limits requests to 64K, but we allow upto 512KB by default for custom SASL mechanisms. In practice," +
|
||||
" PLAIN, SCRAM and OAUTH mechanisms can use much smaller limits.";
|
||||
|
||||
public static final String SSL_ALLOW_DN_CHANGES_CONFIG = "ssl.allow.dn.changes";
|
||||
public static final boolean DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE = false;
|
||||
public static final String SSL_ALLOW_DN_CHANGES_DOC = "Indicates whether changes to the certificate distinguished name should be allowed during" +
|
||||
" a dynamic reconfiguration of certificates or not.";
|
||||
|
||||
public static final String SSL_ALLOW_SAN_CHANGES_CONFIG = "ssl.allow.san.changes";
|
||||
public static final boolean DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE = false;
|
||||
public static final String SSL_ALLOW_SAN_CHANGES_DOC = "Indicates whether changes to the certificate subject alternative names should be allowed during " +
|
||||
"a dynamic reconfiguration of certificates or not.";
|
||||
|
||||
|
|
|
@ -315,7 +315,7 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
|
|||
String prefix = ListenerName.saslMechanismPrefix(mechanism);
|
||||
@SuppressWarnings("unchecked")
|
||||
Class<? extends AuthenticateCallbackHandler> clazz =
|
||||
(Class<? extends AuthenticateCallbackHandler>) configs.get(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS);
|
||||
(Class<? extends AuthenticateCallbackHandler>) configs.get(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG);
|
||||
if (clazz != null)
|
||||
callbackHandler = Utils.newInstance(clazz);
|
||||
else if (mechanism.equals(PlainSaslServer.PLAIN_MECHANISM))
|
||||
|
@ -333,9 +333,9 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
|
|||
private void createConnectionsMaxReauthMsMap(Map<String, ?> configs) {
|
||||
for (String mechanism : jaasContexts.keySet()) {
|
||||
String prefix = ListenerName.saslMechanismPrefix(mechanism);
|
||||
Long connectionsMaxReauthMs = (Long) configs.get(prefix + BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS);
|
||||
Long connectionsMaxReauthMs = (Long) configs.get(prefix + BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG);
|
||||
if (connectionsMaxReauthMs == null)
|
||||
connectionsMaxReauthMs = (Long) configs.get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS);
|
||||
connectionsMaxReauthMs = (Long) configs.get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG);
|
||||
if (connectionsMaxReauthMs != null)
|
||||
connectionsMaxReauthMsByMechanism.put(mechanism, connectionsMaxReauthMs);
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
|||
public class SaslInternalConfigs {
|
||||
/**
|
||||
* The server (broker) specifies a positive session length in milliseconds to a
|
||||
* SASL client when {@link BrokerSecurityConfigs#CONNECTIONS_MAX_REAUTH_MS} is
|
||||
* SASL client when {@link BrokerSecurityConfigs#CONNECTIONS_MAX_REAUTH_MS_CONFIG} is
|
||||
* positive as per <a href=
|
||||
* "https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate">KIP
|
||||
* 368: Allow SASL Connections to Periodically Re-Authenticate</a>. The session
|
||||
|
|
|
@ -182,7 +182,7 @@ public class SaslServerAuthenticator implements Authenticator {
|
|||
throw new IllegalArgumentException("Callback handler not specified for SASL mechanism " + mechanism);
|
||||
if (!subjects.containsKey(mechanism))
|
||||
throw new IllegalArgumentException("Subject cannot be null for SASL mechanism " + mechanism);
|
||||
LOG.trace("{} for mechanism={}: {}", BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, mechanism,
|
||||
LOG.trace("{} for mechanism={}: {}", BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, mechanism,
|
||||
connectionsMaxReauthMsByMechanism.get(mechanism));
|
||||
}
|
||||
|
||||
|
|
|
@ -55,7 +55,7 @@ import org.slf4j.LoggerFactory;
|
|||
*
|
||||
* <p>
|
||||
* This {@link AuthenticateCallbackHandler} is enabled in the broker configuration by setting the
|
||||
* {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}
|
||||
* {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}
|
||||
* like so:
|
||||
*
|
||||
* <code>
|
||||
|
@ -86,7 +86,7 @@ import org.slf4j.LoggerFactory;
|
|||
* validation callback handler:
|
||||
*
|
||||
* <ul>
|
||||
* <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
|
||||
* <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}</li>
|
||||
* <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
|
||||
* <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
|
||||
* <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
|
||||
|
|
|
@ -31,12 +31,12 @@ public class TestSecurityConfig extends AbstractConfig {
|
|||
.define(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, Type.LIST,
|
||||
BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS,
|
||||
Importance.MEDIUM, BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, Type.CLASS,
|
||||
.define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, Type.CLASS,
|
||||
null,
|
||||
Importance.MEDIUM, BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Type.CLASS,
|
||||
null, Importance.MEDIUM, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC)
|
||||
.define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, Type.LONG, 0L, Importance.MEDIUM,
|
||||
.define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, Type.LONG, 0L, Importance.MEDIUM,
|
||||
BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, Type.INT, BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE,
|
||||
Importance.LOW, BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC)
|
||||
|
|
|
@ -187,7 +187,7 @@ public abstract class SaslAuthenticatorFailureDelayTest {
|
|||
saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism);
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverMechanisms);
|
||||
if (serverMechanisms.contains("DIGEST-MD5")) {
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestDigestLoginModule.DigestServerCallbackHandler.class.getName());
|
||||
}
|
||||
return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms);
|
||||
|
|
|
@ -348,7 +348,7 @@ public class SaslAuthenticatorTest {
|
|||
TestJaasConfig jaasConfig = configureMechanisms("SCRAM-SHA-256", Collections.singletonList("SCRAM-SHA-256"));
|
||||
jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_SERVER, PlainLoginModule.class.getName(), new HashMap<>());
|
||||
String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("SCRAM-SHA-256");
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
InvalidScramServerCallbackHandler.class.getName());
|
||||
server = createEchoServer(securityProtocol);
|
||||
|
||||
|
@ -645,7 +645,7 @@ public class SaslAuthenticatorTest {
|
|||
jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_CLIENT, ScramLoginModule.class.getName(), options);
|
||||
|
||||
// ensure re-authentication based on token expiry rather than a default value
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, Long.MAX_VALUE);
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, Long.MAX_VALUE);
|
||||
/*
|
||||
* create a token cache that adjusts the token expiration dynamically so that
|
||||
* the first time the expiry is read during authentication we use it to define a
|
||||
|
@ -1091,7 +1091,7 @@ public class SaslAuthenticatorTest {
|
|||
TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Collections.singletonList("PLAIN"));
|
||||
jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_SERVER, PlainLoginModule.class.getName(), new HashMap<>());
|
||||
String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("PLAIN");
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestServerCallbackHandler.class.getName());
|
||||
server = createEchoServer(securityProtocol);
|
||||
|
||||
|
@ -1116,20 +1116,20 @@ public class SaslAuthenticatorTest {
|
|||
TestJaasConfig jaasConfig = configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5", "PLAIN"));
|
||||
|
||||
// Connections should fail using the digest callback handler if listener.mechanism prefix not specified
|
||||
saslServerConfigs.put("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestServerCallbackHandler.class);
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
DigestServerCallbackHandler.class);
|
||||
server = createEchoServer(securityProtocol);
|
||||
createAndCheckClientConnectionFailure(securityProtocol, "invalid");
|
||||
|
||||
// Connections should succeed using the server callback handler associated with the listener
|
||||
ListenerName listener = ListenerName.forSecurityProtocol(securityProtocol);
|
||||
saslServerConfigs.remove("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS);
|
||||
saslServerConfigs.remove("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS);
|
||||
saslServerConfigs.put(listener.saslMechanismConfigPrefix("plain") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.remove("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG);
|
||||
saslServerConfigs.remove("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG);
|
||||
saslServerConfigs.put(listener.saslMechanismConfigPrefix("plain") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestServerCallbackHandler.class);
|
||||
saslServerConfigs.put(listener.saslMechanismConfigPrefix("digest-md5") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put(listener.saslMechanismConfigPrefix("digest-md5") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
DigestServerCallbackHandler.class);
|
||||
server.close();
|
||||
server = createEchoServer(securityProtocol);
|
||||
|
@ -1218,7 +1218,7 @@ public class SaslAuthenticatorTest {
|
|||
jaasConfig.setClientOptions("PLAIN", TestServerCallbackHandler.USERNAME, TestServerCallbackHandler.PASSWORD);
|
||||
ListenerName listenerName = ListenerName.forSecurityProtocol(securityProtocol);
|
||||
String prefix = listenerName.saslMechanismConfigPrefix("PLAIN");
|
||||
saslServerConfigs.put(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestServerCallbackHandler.class);
|
||||
Class<?> loginCallback = TestLoginCallbackHandler.class;
|
||||
|
||||
|
@ -1757,7 +1757,7 @@ public class SaslAuthenticatorTest {
|
|||
* of the data that the client explicitly sent, and then the client will not
|
||||
* recognize that data and will throw an assertion error.
|
||||
*/
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS,
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG,
|
||||
Double.valueOf(1.1 * 1000L / 0.85).longValue());
|
||||
|
||||
server = createEchoServer(securityProtocol);
|
||||
|
@ -2144,9 +2144,9 @@ public class SaslAuthenticatorTest {
|
|||
private TestJaasConfig configureMechanisms(String clientMechanism, List<String> serverMechanisms) {
|
||||
saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism);
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverMechanisms);
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, CONNECTIONS_MAX_REAUTH_MS_VALUE);
|
||||
saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, CONNECTIONS_MAX_REAUTH_MS_VALUE);
|
||||
if (serverMechanisms.contains("DIGEST-MD5")) {
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestDigestLoginModule.DigestServerCallbackHandler.class.getName());
|
||||
}
|
||||
return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms);
|
||||
|
@ -2154,7 +2154,7 @@ public class SaslAuthenticatorTest {
|
|||
|
||||
private void configureDigestMd5ServerCallback(SecurityProtocol securityProtocol) {
|
||||
String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("DIGEST-MD5");
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS,
|
||||
saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
TestDigestLoginModule.DigestServerCallbackHandler.class);
|
||||
}
|
||||
|
||||
|
|
|
@ -207,7 +207,7 @@ public abstract class RestServerConfig extends AbstractConfig {
|
|||
).define(
|
||||
BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
||||
ConfigDef.Type.STRING,
|
||||
SslClientAuth.NONE.toString(),
|
||||
BrokerSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT,
|
||||
in(Utils.enumOptions(SslClientAuth.class)),
|
||||
ConfigDef.Importance.LOW,
|
||||
BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC);
|
||||
|
|
|
@ -29,7 +29,8 @@ import kafka.utils.{CoreUtils, Logging}
|
|||
import kafka.utils.Implicits._
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.common.Reconfigurable
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SslConfigs}
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs}
|
||||
import org.apache.kafka.common.metrics.{JmxReporter, Metrics, MetricsReporter}
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable}
|
||||
|
@ -39,7 +40,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
|
|||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.security.PasswordEncoder
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals}
|
||||
import org.apache.kafka.server.config.{ConfigType, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs}
|
||||
import org.apache.kafka.server.telemetry.ClientTelemetry
|
||||
|
@ -101,11 +102,11 @@ object DynamicBrokerConfig {
|
|||
private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, ServerConfigs.NUM_NETWORK_THREADS_CONFIG)
|
||||
private val PerBrokerConfigs = (DynamicSecurityConfigs ++ DynamicListenerConfig.ReconfigurableConfigs).diff(
|
||||
ClusterLevelListenerConfigs)
|
||||
private val ListenerMechanismConfigs = Set(KafkaSecurityConfigs.SASL_JAAS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG)
|
||||
private val ListenerMechanismConfigs = Set(SaslConfigs.SASL_JAAS_CONFIG,
|
||||
SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS,
|
||||
SaslConfigs.SASL_LOGIN_CLASS,
|
||||
BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG,
|
||||
BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG)
|
||||
|
||||
private val ReloadableFileConfigs = Set(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)
|
||||
|
||||
|
@ -970,39 +971,39 @@ object DynamicListenerConfig {
|
|||
SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG,
|
||||
|
||||
// SSL configs
|
||||
KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_PROVIDER_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
||||
KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG,
|
||||
BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG,
|
||||
SslConfigs.SSL_PROTOCOL_CONFIG,
|
||||
SslConfigs.SSL_PROVIDER_CONFIG,
|
||||
SslConfigs.SSL_CIPHER_SUITES_CONFIG,
|
||||
SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_TYPE_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG,
|
||||
SslConfigs.SSL_KEY_PASSWORD_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG,
|
||||
SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG,
|
||||
SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG,
|
||||
SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG,
|
||||
SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG,
|
||||
BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
||||
SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG,
|
||||
|
||||
// SASL configs
|
||||
KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_JAAS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG,
|
||||
KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG,
|
||||
BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG,
|
||||
SaslConfigs.SASL_JAAS_CONFIG,
|
||||
BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG,
|
||||
SaslConfigs.SASL_KERBEROS_SERVICE_NAME,
|
||||
SaslConfigs.SASL_KERBEROS_KINIT_CMD,
|
||||
SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR,
|
||||
SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER,
|
||||
SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN,
|
||||
BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG,
|
||||
SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR,
|
||||
SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER,
|
||||
SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS,
|
||||
SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS,
|
||||
|
||||
// Connection limit configs
|
||||
SocketServerConfigs.MAX_CONNECTIONS_CONFIG,
|
||||
|
|
|
@ -25,8 +25,9 @@ import kafka.utils.{CoreUtils, Logging}
|
|||
import kafka.utils.Implicits._
|
||||
import org.apache.kafka.common.Reconfigurable
|
||||
import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList}
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslClientAuth, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.config.ConfigDef.{CaseInsensitiveValidString, ConfigKey, ValidList, ValidString}
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.record.{CompressionType, LegacyRecord, Records, TimestampType}
|
||||
|
@ -46,7 +47,7 @@ import org.apache.kafka.server.ProcessRole
|
|||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.server.metrics.MetricConfigs
|
||||
import org.apache.kafka.server.record.BrokerCompressionType
|
||||
|
@ -96,7 +97,6 @@ object KafkaConfig {
|
|||
import ConfigDef.Importance._
|
||||
import ConfigDef.Range._
|
||||
import ConfigDef.Type._
|
||||
import ConfigDef.ValidString._
|
||||
|
||||
new ConfigDef()
|
||||
|
||||
|
@ -227,7 +227,7 @@ object KafkaConfig {
|
|||
.define(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DEFAULT, HIGH, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DOC)
|
||||
.define(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ConfigDef.ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC)
|
||||
.define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
|
||||
|
@ -257,7 +257,7 @@ object KafkaConfig {
|
|||
.define(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG, INT, ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DEFAULT, HIGH, ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DOC)
|
||||
.define(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG, LONG, ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DEFAULT, atLeast(1), HIGH, ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DOC)
|
||||
.define(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_DOC)
|
||||
.define(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DEFAULT, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DOC)
|
||||
.define(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DEFAULT, ValidString.in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DOC)
|
||||
.define(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_DOC)
|
||||
.define(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, STRING, null, MEDIUM, ReplicationConfigs.INTER_BROKER_LISTENER_NAME_DOC)
|
||||
.define(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG, STRING, null, MEDIUM, ReplicationConfigs.REPLICA_SELECTOR_CLASS_DOC)
|
||||
|
@ -276,7 +276,7 @@ object KafkaConfig {
|
|||
|
||||
/** New group coordinator configs */
|
||||
.define(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT,
|
||||
ConfigDef.ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC)
|
||||
ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC)
|
||||
.define(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DOC)
|
||||
// Internal configuration used by integration and system tests.
|
||||
.defineInternal(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, BOOLEAN, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DOC)
|
||||
|
@ -290,7 +290,7 @@ object KafkaConfig {
|
|||
.define(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC)
|
||||
.define(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG, INT, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_DOC)
|
||||
.define(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_DOC)
|
||||
.defineInternal(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(classOf[ConsumerGroupMigrationPolicy]): _*), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DOC)
|
||||
.defineInternal(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, CaseInsensitiveValidString.in(Utils.enumOptions(classOf[ConsumerGroupMigrationPolicy]): _*), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DOC)
|
||||
|
||||
/** ********* Offset management configuration ***********/
|
||||
.define(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG, INT, GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_DOC)
|
||||
|
@ -304,7 +304,7 @@ object KafkaConfig {
|
|||
.define(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DOC)
|
||||
.define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC)
|
||||
.define(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, BOOLEAN, ServerConfigs.DELETE_TOPIC_ENABLE_DEFAULT, HIGH, ServerConfigs.DELETE_TOPIC_ENABLE_DOC)
|
||||
.define(ServerConfigs.COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, ServerConfigs.COMPRESSION_TYPE_DOC)
|
||||
.define(ServerConfigs.COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, ValidString.in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, ServerConfigs.COMPRESSION_TYPE_DOC)
|
||||
.define(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, ServerConfigs.COMPRESSION_GZIP_LEVEL_DOC)
|
||||
.define(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_LZ4_LEVEL_DOC)
|
||||
.define(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_ZSTD_LEVEL_DOC)
|
||||
|
@ -359,68 +359,68 @@ object KafkaConfig {
|
|||
.define(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, CLASS, null, LOW, QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_DOC)
|
||||
|
||||
/** ********* General Security Configuration ****************/
|
||||
.define(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, LONG, KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DEFAULT, MEDIUM, KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, INT, KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC)
|
||||
.define(KafkaSecurityConfigs.SECURITY_PROVIDER_CLASS_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SECURITY_PROVIDERS_DOC)
|
||||
.define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, LONG, BrokerSecurityConfigs.DEFAULT_CONNECTIONS_MAX_REAUTH_MS, MEDIUM, BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, INT, BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE, MEDIUM, BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC)
|
||||
.define(SecurityConfig.SECURITY_PROVIDERS_CONFIG, STRING, null, LOW, SecurityConfig.SECURITY_PROVIDERS_DOC)
|
||||
|
||||
/** ********* SSL Configuration ****************/
|
||||
.define(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, CLASS, KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DEFAULT, MEDIUM, KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG, STRING, KafkaSecurityConfigs.SSL_PROTOCOL_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_PROTOCOL_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_PROVIDER_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_DEFAULTS, MEDIUM, KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEY_PASSWORD_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_KEY_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, STRING, KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, STRING, KafkaSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT, in(KafkaSecurityConfigs.SSL_CLIENT_AUTHENTICATION_VALID_VALUES:_*), MEDIUM, KafkaSecurityConfigs.SSL_CLIENT_AUTH_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG, LIST, Collections.emptyList(), MEDIUM, KafkaSecurityConfigs.SSL_CIPHER_SUITES_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG, STRING, KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, CLASS, null, LOW, KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG, BOOLEAN, KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC)
|
||||
.define(KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC)
|
||||
.define(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, CLASS, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DEFAULT, MEDIUM, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC)
|
||||
.define(SslConfigs.SSL_PROTOCOL_CONFIG, STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, MEDIUM, SslConfigs.SSL_PROTOCOL_DOC)
|
||||
.define(SslConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_PROVIDER_DOC)
|
||||
.define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC)
|
||||
.define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC)
|
||||
.define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_KEYSTORE_LOCATION_DOC)
|
||||
.define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC)
|
||||
.define(SslConfigs.SSL_KEY_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEY_PASSWORD_DOC)
|
||||
.define(SslConfigs.SSL_KEYSTORE_KEY_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_KEY_DOC)
|
||||
.define(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC)
|
||||
.define(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, MEDIUM, SslConfigs.SSL_TRUSTSTORE_TYPE_DOC)
|
||||
.define(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_LOCATION_DOC)
|
||||
.define(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC)
|
||||
.define(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC)
|
||||
.define(SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, MEDIUM, SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
|
||||
.define(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, MEDIUM, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
|
||||
.define(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC)
|
||||
.define(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, STRING, null, LOW, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC)
|
||||
.define(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, STRING, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT, ValidString.in(Utils.enumOptions(classOf[SslClientAuth]):_*), MEDIUM, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC)
|
||||
.define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, LIST, Collections.emptyList(), MEDIUM, SslConfigs.SSL_CIPHER_SUITES_DOC)
|
||||
.define(BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG, STRING, BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES, LOW, BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC)
|
||||
.define(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, CLASS, null, LOW, SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC)
|
||||
.define(BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC)
|
||||
.define(BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC)
|
||||
|
||||
/** ********* Sasl Configuration ****************/
|
||||
.define(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, STRING, KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_JAAS_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SASL_JAAS_CONFIG_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, LIST, KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG, STRING, KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG, LONG, KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, LIST, KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG, SHORT, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG, SHORT, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG, INT, null, LOW, KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_CONFIG, INT, null, LOW, KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG, STRING, KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG, STRING, KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG, INT, KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG, LIST, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
|
||||
.define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, MEDIUM, BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC)
|
||||
.define(SaslConfigs.SASL_JAAS_CONFIG, PASSWORD, null, MEDIUM, SaslConfigs.SASL_JAAS_CONFIG_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, LIST, BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS, MEDIUM, BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_LOGIN_CLASS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC)
|
||||
.define(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, STRING, null, MEDIUM, SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC)
|
||||
.define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, MEDIUM, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
|
||||
.define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, MEDIUM, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
|
||||
.define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, MEDIUM, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
|
||||
.define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, MEDIUM, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
|
||||
.define(BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, LIST, BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES, MEDIUM, BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR, DOUBLE, SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER, DOUBLE, SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, SHORT, SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS, SHORT, SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS, INT, null, LOW, SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS, INT, null, LOW, SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC)
|
||||
.define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, LOW, SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
|
||||
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, STRING, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC)
|
||||
|
||||
/** ********* Delegation Token Configuration ****************/
|
||||
.define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC)
|
||||
|
@ -996,7 +996,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
// Hence the base SSL/SASL configs are not fields of KafkaConfig, listener configs should be
|
||||
// retrieved using KafkaConfig#valuesWithPrefixOverride
|
||||
private def saslEnabledMechanisms(listenerName: ListenerName): Set[String] = {
|
||||
val value = valuesWithPrefixOverride(listenerName.configPrefix).get(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG)
|
||||
val value = valuesWithPrefixOverride(listenerName.configPrefix).get(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG)
|
||||
if (value != null)
|
||||
value.asInstanceOf[util.List[String]].asScala.toSet
|
||||
else
|
||||
|
@ -1007,7 +1007,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
def interBrokerSecurityProtocol = getInterBrokerListenerNameAndSecurityProtocol._2
|
||||
def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol.map { case (listenerName, _) => listenerName }
|
||||
def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol.map { case (_, securityProtocol) => securityProtocol }
|
||||
def saslMechanismInterBrokerProtocol = getString(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG)
|
||||
def saslMechanismInterBrokerProtocol = getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG)
|
||||
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled
|
||||
|
||||
/** ********* DelegationToken Configuration **************/
|
||||
|
@ -1408,7 +1408,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
|
||||
s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")
|
||||
require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol),
|
||||
s"${KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication")
|
||||
s"${BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication")
|
||||
require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes,
|
||||
s"${ServerConfigs.QUEUED_MAX_BYTES_CONFIG} must be larger or equal to ${SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG}")
|
||||
|
||||
|
@ -1426,10 +1426,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
s" ${SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG}=$connectionsMaxIdleMs to prevent failed" +
|
||||
s" authentication responses from timing out")
|
||||
|
||||
val principalBuilderClass = getClass(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)
|
||||
require(principalBuilderClass != null, s"${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must be non-null")
|
||||
val principalBuilderClass = getClass(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)
|
||||
require(principalBuilderClass != null, s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must be non-null")
|
||||
require(classOf[KafkaPrincipalSerde].isAssignableFrom(principalBuilderClass),
|
||||
s"${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde")
|
||||
s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde")
|
||||
|
||||
// New group coordinator configs validation.
|
||||
require(consumerGroupMaxHeartbeatIntervalMs >= consumerGroupMinHeartbeatIntervalMs,
|
||||
|
|
|
@ -21,11 +21,11 @@ import kafka.utils.TestUtils.assertFutureExceptionTypeEquals
|
|||
import kafka.utils.{Logging, TestUtils}
|
||||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, AlterConfigsOptions, Config, ConfigEntry}
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.config.{ServerConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.policy.AlterConfigPolicy
|
||||
import org.apache.kafka.storage.internals.log.LogConfig
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
|
||||
|
@ -142,7 +142,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
|||
|
||||
val topicConfigEntries3 = Seq(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "-1")).asJava
|
||||
|
||||
val brokerConfigEntries = Seq(new ConfigEntry(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "12313")).asJava
|
||||
val brokerConfigEntries = Seq(new ConfigEntry(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "12313")).asJava
|
||||
|
||||
// Alter configs: second is valid, the others are invalid
|
||||
var alterResult = client.alterConfigs(Map(
|
||||
|
@ -172,7 +172,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
|||
|
||||
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
|
||||
|
||||
assertNull(configs.get(brokerResource).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
assertNull(configs.get(brokerResource).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
|
||||
// Alter configs with validateOnly = true: only second is valid
|
||||
topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.7")).asJava
|
||||
|
@ -204,7 +204,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
|||
|
||||
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
|
||||
|
||||
assertNull(configs.get(brokerResource).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
assertNull(configs.get(brokerResource).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
|
||||
// Do an incremental alter config on the broker, ensure we don't see the broker config we set earlier in the policy
|
||||
alterResult = client.incrementalAlterConfigs(Map(
|
||||
|
|
|
@ -24,12 +24,13 @@ import kafka.utils.TestUtils._
|
|||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription}
|
||||
import org.apache.kafka.common.Uuid
|
||||
import org.apache.kafka.common.acl.AclOperation
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
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.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.security.authorizer.AclEntry
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
|
||||
|
||||
|
@ -208,8 +209,8 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
|
|||
config.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false")
|
||||
// We set this in order to test that we don't expose sensitive data via describe configs. This will already be
|
||||
// set for subclasses with security enabled and we don't want to overwrite it.
|
||||
if (!config.containsKey(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG))
|
||||
config.setProperty(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "some.invalid.pass")
|
||||
if (!config.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG))
|
||||
config.setProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "some.invalid.pass")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,9 +29,10 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig}
|
|||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||
import org.apache.kafka.common.{Cluster, Reconfigurable}
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs}
|
||||
import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs}
|
||||
import org.apache.kafka.server.quota._
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
|
@ -63,7 +64,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup {
|
|||
override def setUp(testInfo: TestInfo): Unit = {
|
||||
startSasl(jaasSections(kafkaServerSaslMechanisms, Some("SCRAM-SHA-256"), KafkaSasl, JaasTestUtils.KafkaServerContextName))
|
||||
this.serverConfig.setProperty(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, classOf[GroupedUserQuotaCallback].getName)
|
||||
this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}",
|
||||
this.serverConfig.setProperty(s"${listenerName.configPrefix}${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}",
|
||||
classOf[GroupedUserPrincipalBuilder].getName)
|
||||
this.serverConfig.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true")
|
||||
super.setUp(testInfo)
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
|
|||
import org.apache.kafka.common.acl._
|
||||
import org.apache.kafka.common.acl.AclOperation._
|
||||
import org.apache.kafka.common.acl.AclPermissionType._
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
||||
import org.apache.kafka.common.errors.{GroupAuthorizationException, TopicAuthorizationException}
|
||||
import org.apache.kafka.common.resource._
|
||||
|
@ -38,7 +39,7 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
|
|||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
|
||||
|
@ -137,7 +138,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3")
|
||||
this.serverConfig.setProperty(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3")
|
||||
this.serverConfig.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "3")
|
||||
this.serverConfig.setProperty(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500")
|
||||
this.serverConfig.setProperty(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500")
|
||||
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group")
|
||||
this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1500")
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.kafka.clients.admin._
|
|||
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType}
|
||||
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.errors._
|
||||
import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse}
|
||||
import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
|
||||
|
@ -46,7 +46,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX
|
|||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.security.authorizer.AclEntry
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
|
||||
import org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
@ -433,8 +433,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
assertFalse(listenerSecurityProtocolMap.isDefault)
|
||||
assertFalse(listenerSecurityProtocolMap.isSensitive)
|
||||
assertFalse(listenerSecurityProtocolMap.isReadOnly)
|
||||
val truststorePassword = configs.get(brokerResource1).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)
|
||||
assertEquals(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststorePassword.name)
|
||||
val truststorePassword = configs.get(brokerResource1).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)
|
||||
assertEquals(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststorePassword.name)
|
||||
assertNull(truststorePassword.value)
|
||||
assertFalse(truststorePassword.isDefault)
|
||||
assertTrue(truststorePassword.isSensitive)
|
||||
|
|
|
@ -19,9 +19,8 @@ package kafka.api
|
|||
import kafka.security.authorizer.AclAuthorizer
|
||||
import kafka.utils.JaasTestUtils
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
|
||||
import org.junit.jupiter.api.Assertions.assertNull
|
||||
|
||||
import scala.collection.immutable.List
|
||||
|
@ -39,8 +38,8 @@ class SaslGssapiSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTe
|
|||
// Configure brokers to require SSL client authentication in order to verify that SASL_SSL works correctly even if the
|
||||
// client doesn't have a keystore. We want to cover the scenario where a broker requires either SSL client
|
||||
// authentication or SASL authentication with SSL as the transport layer (but not both).
|
||||
serverConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
controllerConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
serverConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
controllerConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
assertNull(producerConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertNull(consumerConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertNull(adminClientConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.kafka.common.network.Mode
|
|||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
||||
import org.apache.kafka.common.security.plain.PlainAuthenticateCallback
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -111,11 +110,11 @@ object SaslPlainSslEndToEndAuthorizationTest {
|
|||
class SaslPlainSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest {
|
||||
import SaslPlainSslEndToEndAuthorizationTest._
|
||||
|
||||
this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG}", "required")
|
||||
this.serverConfig.setProperty(s"${listenerName.configPrefix}${BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG}", "required")
|
||||
this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[TestPrincipalBuilder].getName)
|
||||
this.serverConfig.put(KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG, classOf[TestClientCallbackHandler].getName)
|
||||
this.serverConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName)
|
||||
val mechanismPrefix = listenerName.saslMechanismConfigPrefix("PLAIN")
|
||||
this.serverConfig.put(s"$mechanismPrefix${KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}", classOf[TestServerCallbackHandler].getName)
|
||||
this.serverConfig.put(s"$mechanismPrefix${BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}", classOf[TestServerCallbackHandler].getName)
|
||||
this.producerConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName)
|
||||
this.consumerConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName)
|
||||
this.adminClientConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName)
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
|
|||
import org.apache.kafka.common.security.authenticator.LoginManager
|
||||
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs}
|
||||
import org.apache.kafka.server.config.ConfigType
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
import scala.util.Using
|
||||
|
@ -133,7 +133,7 @@ trait SaslSetup {
|
|||
|
||||
def kafkaServerSaslProperties(serverSaslMechanisms: Seq[String], interBrokerSaslMechanism: String): Properties = {
|
||||
val props = new Properties
|
||||
props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, interBrokerSaslMechanism)
|
||||
props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, interBrokerSaslMechanism)
|
||||
props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverSaslMechanisms.mkString(","))
|
||||
props
|
||||
}
|
||||
|
|
|
@ -16,9 +16,9 @@ package kafka.api
|
|||
|
||||
import kafka.server._
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||
import org.apache.kafka.common.utils.Sanitizer
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
||||
|
||||
class UserClientIdQuotaTest extends BaseQuotaTest {
|
||||
|
@ -31,7 +31,7 @@ class UserClientIdQuotaTest extends BaseQuotaTest {
|
|||
|
||||
@BeforeEach
|
||||
override def setUp(testInfo: TestInfo): Unit = {
|
||||
this.serverConfig.setProperty(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
this.serverConfig.setProperty(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
super.setUp(testInfo)
|
||||
quotaTestClients.alterClientQuotas(
|
||||
quotaTestClients.clientQuotaAlteration(
|
||||
|
|
|
@ -44,8 +44,9 @@ import org.apache.kafka.clients.admin._
|
|||
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecord, ConsumerRecords, KafkaConsumer}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||
import org.apache.kafka.common.{ClusterResource, ClusterResourceListener, Reconfigurable, TopicPartition, TopicPartitionInfo}
|
||||
import org.apache.kafka.common.config.{ConfigException, ConfigResource}
|
||||
import org.apache.kafka.common.config.{ConfigException, ConfigResource, SaslConfigs}
|
||||
import org.apache.kafka.common.config.SslConfigs._
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.config.provider.FileConfigProvider
|
||||
import org.apache.kafka.common.errors.{AuthenticationException, InvalidRequestException}
|
||||
|
@ -62,7 +63,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ
|
|||
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.security.{PasswordEncoder, PasswordEncoderConfigs}
|
||||
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{ConfigType, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs}
|
||||
import org.apache.kafka.server.record.BrokerCompressionType
|
||||
import org.apache.kafka.server.util.ShutdownableThread
|
||||
|
@ -132,9 +133,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
|||
props.put(SocketServerConfigs.LISTENERS_CONFIG, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0")
|
||||
props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"PLAINTEXT:PLAINTEXT, $SecureInternal:SSL, $SecureExternal:SASL_SSL, CONTROLLER:$controllerListenerSecurityProtocol")
|
||||
props.put(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, SecureInternal)
|
||||
props.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested")
|
||||
props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN")
|
||||
props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(","))
|
||||
props.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested")
|
||||
props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN")
|
||||
props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(","))
|
||||
props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "2000") // low value to test log rolling on config update
|
||||
props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") // greater than one to test reducing threads
|
||||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "dynamic-config-secret")
|
||||
|
@ -1743,12 +1744,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
|||
private def addListenerPropsSasl(listener: String, mechanisms: Seq[String], props: Properties): Unit = {
|
||||
val listenerName = new ListenerName(listener)
|
||||
val prefix = listenerName.configPrefix
|
||||
props.put(prefix + KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, mechanisms.mkString(","))
|
||||
props.put(prefix + KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, "kafka")
|
||||
props.put(prefix + BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, mechanisms.mkString(","))
|
||||
props.put(prefix + SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
mechanisms.foreach { mechanism =>
|
||||
val jaasSection = jaasSections(Seq(mechanism), None, KafkaSasl, "").head
|
||||
val jaasConfig = jaasSection.modules.head.toString
|
||||
props.put(listenerName.saslMechanismConfigPrefix(mechanism) + KafkaSecurityConfigs.SASL_JAAS_CONFIG, jaasConfig)
|
||||
props.put(listenerName.saslMechanismConfigPrefix(mechanism) + SaslConfigs.SASL_JAAS_CONFIG, jaasConfig)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -28,6 +28,7 @@ import kafka.utils.TestUtils
|
|||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.errors.SaslAuthenticationException
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.network._
|
||||
|
@ -36,7 +37,6 @@ import org.apache.kafka.common.security.auth.{Login, SecurityProtocol}
|
|||
import org.apache.kafka.common.security.kerberos.KerberosLogin
|
||||
import org.apache.kafka.common.utils.{LogContext, MockTime}
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
|
||||
|
@ -63,7 +63,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
|
|||
override def setUp(testInfo: TestInfo): Unit = {
|
||||
TestableKerberosLogin.reset()
|
||||
startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both))
|
||||
serverConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
serverConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required")
|
||||
serverConfig.put(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, failedAuthenticationDelayMs.toString)
|
||||
super.setUp(testInfo)
|
||||
serverAddr = new InetSocketAddress("localhost",
|
||||
|
|
|
@ -26,10 +26,11 @@ import kafka.utils.{JaasTestUtils, TestUtils}
|
|||
import kafka.utils.Implicits._
|
||||
import org.apache.kafka.clients.consumer.Consumer
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.{SaslConfigs, SslConfigs}
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.network.{ListenerName, Mode}
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ReplicationConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{ReplicationConfigs, ZkConfigs}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
|
@ -82,12 +83,12 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
|
|||
s"$External:PLAINTEXT, $SecureExternal:SASL_SSL")
|
||||
props.put(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, Internal)
|
||||
props.put(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true")
|
||||
props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism)
|
||||
props.put(s"${new ListenerName(SecureInternal).configPrefix}${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}",
|
||||
props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism)
|
||||
props.put(s"${new ListenerName(SecureInternal).configPrefix}${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}",
|
||||
kafkaServerSaslMechanisms(SecureInternal).mkString(","))
|
||||
props.put(s"${new ListenerName(SecureExternal).configPrefix}${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}",
|
||||
props.put(s"${new ListenerName(SecureExternal).configPrefix}${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}",
|
||||
kafkaServerSaslMechanisms(SecureExternal).mkString(","))
|
||||
props.put(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, "kafka")
|
||||
props.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
props ++= dynamicJaasSections
|
||||
|
||||
props ++= TestUtils.sslConfigs(Mode.SERVER, clientCert = false, Some(trustStoreFile), s"server$brokerId")
|
||||
|
@ -176,7 +177,7 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
|
|||
val listenerName = new ListenerName(listener)
|
||||
val prefix = listenerName.saslMechanismConfigPrefix(mechanism)
|
||||
val jaasConfig = jaasSection.modules.head.toString
|
||||
props.put(s"${prefix}${KafkaSecurityConfigs.SASL_JAAS_CONFIG}", jaasConfig)
|
||||
props.put(s"${prefix}${SaslConfigs.SASL_JAAS_CONFIG}", jaasConfig)
|
||||
}
|
||||
|
||||
case class ClientMetadata(listenerName: ListenerName, saslMechanism: String, topic: String) {
|
||||
|
|
|
@ -22,13 +22,13 @@ import java.util.Properties
|
|||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{Exit, TestUtils}
|
||||
import kafka.utils.TestUtils.assertBadConfigContainingMessage
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.internals.FatalExitError
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
@ -198,19 +198,19 @@ class KafkaConfigTest {
|
|||
"--override", "ssl.keystore.certificate.chain=certificate_chain",
|
||||
"--override", "ssl.keystore.key=private_key",
|
||||
"--override", "ssl.truststore.certificates=truststore_certificates")))
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_KEY_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).toString)
|
||||
|
||||
assertEquals("key_password", config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).value)
|
||||
assertEquals("keystore_password", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals("truststore_password", config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals("private_key", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG).value)
|
||||
assertEquals("certificate_chain", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).value)
|
||||
assertEquals("truststore_certificates", config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).value)
|
||||
assertEquals("key_password", config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).value)
|
||||
assertEquals("keystore_password", config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals("truststore_password", config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals("private_key", config.getPassword(SslConfigs.SSL_KEYSTORE_KEY_CONFIG).value)
|
||||
assertEquals("certificate_chain", config.getPassword(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).value)
|
||||
assertEquals("truststore_certificates", config.getPassword(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).value)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -221,13 +221,13 @@ class KafkaConfigTest {
|
|||
"--override", "ssl.keystore.password=" + password,
|
||||
"--override", "ssl.key.password=" + password,
|
||||
"--override", "ssl.truststore.password=" + password)))
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString)
|
||||
assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString)
|
||||
|
||||
assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).value)
|
||||
assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals(password, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value)
|
||||
assertEquals(password, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).value)
|
||||
assertEquals(password, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value)
|
||||
}
|
||||
|
||||
private val booleanPropValueToSet = true
|
||||
|
@ -346,7 +346,7 @@ class KafkaConfigTest {
|
|||
def testConnectionsMaxReauthMsDefault(): Unit = {
|
||||
val propertiesFile = prepareDefaultConfig()
|
||||
val config = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile)))
|
||||
assertEquals(0L, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS).asInstanceOf[Long])
|
||||
assertEquals(0L, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG).asInstanceOf[Long])
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -354,7 +354,7 @@ class KafkaConfigTest {
|
|||
val propertiesFile = prepareDefaultConfig()
|
||||
val expected = 3600000
|
||||
val config = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", s"sasl_ssl.oauthbearer.connections.max.reauth.ms=$expected")))
|
||||
assertEquals(expected, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS).asInstanceOf[Long])
|
||||
assertEquals(expected, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG).asInstanceOf[Long])
|
||||
}
|
||||
|
||||
private def testZkConfig[T, U](kafkaPropName: String,
|
||||
|
|
|
@ -21,13 +21,13 @@ import kafka.server.KafkaConfig
|
|||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM
|
||||
import org.apache.kafka.common.config.ConfigDef.Type.INT
|
||||
import org.apache.kafka.common.config.{ConfigException, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.util.{Collections, Properties}
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.config.ServerLogConfigs
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -188,7 +188,7 @@ class LogConfigTest {
|
|||
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
|
||||
kafkaProps.put("unknown.broker.password.config", "aaaaa")
|
||||
kafkaProps.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "50")
|
||||
kafkaProps.put(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword")
|
||||
kafkaProps.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword")
|
||||
val kafkaConfig = KafkaConfig.fromProps(kafkaProps)
|
||||
val topicOverrides = new Properties
|
||||
// Only set as a topic config
|
||||
|
@ -196,7 +196,7 @@ class LogConfigTest {
|
|||
// Overrides value from broker config
|
||||
topicOverrides.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "100")
|
||||
// Unknown topic config, but known broker config
|
||||
topicOverrides.setProperty(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "sometrustpasswrd")
|
||||
topicOverrides.setProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "sometrustpasswrd")
|
||||
// Unknown config
|
||||
topicOverrides.setProperty("unknown.topic.password.config", "bbbb")
|
||||
// We don't currently have any sensitive topic configs, if we add them, we should set one here
|
||||
|
|
|
@ -27,6 +27,7 @@ import kafka.zk.KafkaZkClient
|
|||
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclOperation}
|
||||
import org.apache.kafka.common.acl.AclOperation._
|
||||
import org.apache.kafka.common.acl.AclPermissionType.ALLOW
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.resource.PatternType.LITERAL
|
||||
import org.apache.kafka.common.resource.ResourcePattern
|
||||
|
@ -40,7 +41,7 @@ import org.apache.kafka.network.Session
|
|||
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
|
||||
import org.apache.kafka.security.authorizer.AuthorizerUtils
|
||||
import org.apache.kafka.server.authorizer._
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KafkaSecurityConfigs}
|
||||
import org.apache.kafka.server.config.DelegationTokenManagerConfigs
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
|
||||
|
@ -72,7 +73,7 @@ class DelegationTokenManagerTest extends QuorumTestHarness {
|
|||
override def setUp(testInfo: TestInfo): Unit = {
|
||||
super.setUp(testInfo)
|
||||
props = TestUtils.createBrokerConfig(0, zkConnect, enableToken = true)
|
||||
props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(","))
|
||||
props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(","))
|
||||
props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, secretKey)
|
||||
tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames())
|
||||
}
|
||||
|
|
|
@ -18,11 +18,12 @@ package kafka.server
|
|||
|
||||
import kafka.network.SocketServer
|
||||
import org.apache.kafka.clients.admin.ScramMechanism
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse}
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs}
|
||||
import org.apache.kafka.server.config.ServerConfigs
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -38,7 +39,7 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest
|
|||
override def brokerPropertyOverrides(properties: Properties): Unit = {
|
||||
properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false")
|
||||
properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName)
|
||||
properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
||||
}
|
||||
|
||||
private val user1 = "user1"
|
||||
|
|
|
@ -25,6 +25,7 @@ import kafka.network.SocketServer
|
|||
import kafka.security.authorizer.AclAuthorizer
|
||||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||
import org.apache.kafka.clients.admin.ScramMechanism
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult
|
||||
import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
|
||||
|
@ -34,7 +35,7 @@ import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrinci
|
|||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs}
|
||||
import org.apache.kafka.server.config.ServerConfigs
|
||||
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -63,7 +64,7 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
|||
this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName)
|
||||
|
||||
}
|
||||
this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName)
|
||||
this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName)
|
||||
this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false")
|
||||
|
||||
super.setUp(testInfo)
|
||||
|
|
|
@ -18,6 +18,7 @@ import java.util.concurrent.ExecutionException
|
|||
import java.util.concurrent.TimeUnit
|
||||
import kafka.server.ClientQuotaManager.DefaultTags
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
|
@ -40,7 +41,7 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse
|
|||
import org.apache.kafka.common.security.auth.AuthenticationContext
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs}
|
||||
import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
|
@ -98,7 +99,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
|||
properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false")
|
||||
properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
|
||||
properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1")
|
||||
properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG,
|
||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG,
|
||||
classOf[ControllerMutationQuotaTest.TestPrincipalBuilder].getName)
|
||||
// Specify number of samples and window size.
|
||||
properties.put(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG, ControllerQuotaSamples.toString)
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
package kafka.server
|
||||
|
||||
import kafka.network.SocketServer
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
|
||||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs}
|
||||
import org.apache.kafka.server.config.ServerConfigs
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
@ -39,7 +40,7 @@ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTe
|
|||
} else {
|
||||
properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[DescribeCredentialsTest.TestAuthorizer].getName)
|
||||
}
|
||||
properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.util
|
|||
import kafka.utils.TestInfoUtils
|
||||
import kafka.network.SocketServer
|
||||
import kafka.security.authorizer.AclAuthorizer
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||
import org.apache.kafka.common.message.{DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData}
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName
|
||||
|
@ -28,7 +29,7 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De
|
|||
import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal}
|
||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs}
|
||||
import org.apache.kafka.server.config.ServerConfigs
|
||||
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -50,7 +51,7 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
|
|||
this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName)
|
||||
|
||||
}
|
||||
this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName)
|
||||
this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName)
|
||||
this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false")
|
||||
|
||||
super.setUp(testInfo)
|
||||
|
|
|
@ -30,14 +30,14 @@ import kafka.zk.KafkaZkClient
|
|||
import org.apache.kafka.common.{Endpoint, Reconfigurable}
|
||||
import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter}
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.config.{ConfigException, SslConfigs}
|
||||
import org.apache.kafka.common.config.{ConfigException, SaslConfigs, SslConfigs}
|
||||
import org.apache.kafka.common.metrics.{JmxReporter, Metrics}
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.security.PasswordEncoderConfigs
|
||||
import org.apache.kafka.server.authorizer._
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs}
|
||||
import org.apache.kafka.server.util.KafkaScheduler
|
||||
|
@ -86,7 +86,7 @@ class DynamicBrokerConfigTest {
|
|||
assertEquals(newKeystore,
|
||||
config.originalsWithPrefix("listener.name.external.").get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
|
||||
assertEquals(oldKeystore, config.getString(KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertEquals(oldKeystore, config.getString(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertEquals(oldKeystore, config.originals.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertEquals(oldKeystore, config.values.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
assertEquals(oldKeystore, config.originalsStrings.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG))
|
||||
|
@ -386,7 +386,7 @@ class DynamicBrokerConfigTest {
|
|||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "config-encoder-secret")
|
||||
val configWithSecret = KafkaConfig(props)
|
||||
val dynamicProps = new Properties
|
||||
dynamicProps.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "myLoginModule required;")
|
||||
dynamicProps.put(SaslConfigs.SASL_JAAS_CONFIG, "myLoginModule required;")
|
||||
|
||||
try {
|
||||
configWithoutSecret.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true)
|
||||
|
@ -394,46 +394,46 @@ class DynamicBrokerConfigTest {
|
|||
case _: ConfigException => // expected exception
|
||||
}
|
||||
val persistedProps = configWithSecret.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true)
|
||||
assertFalse(persistedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG).contains("myLoginModule"),
|
||||
assertFalse(persistedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG).contains("myLoginModule"),
|
||||
"Password not encoded")
|
||||
val decodedProps = configWithSecret.dynamicConfig.fromPersistentProps(persistedProps, perBrokerConfig = true)
|
||||
assertEquals("myLoginModule required;", decodedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG))
|
||||
assertEquals("myLoginModule required;", decodedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPasswordConfigEncoderSecretChange(): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
|
||||
props.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "staticLoginModule required;")
|
||||
props.put(SaslConfigs.SASL_JAAS_CONFIG, "staticLoginModule required;")
|
||||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "config-encoder-secret")
|
||||
val config = KafkaConfig(props)
|
||||
config.dynamicConfig.initialize(None, None)
|
||||
val dynamicProps = new Properties
|
||||
dynamicProps.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "dynamicLoginModule required;")
|
||||
dynamicProps.put(SaslConfigs.SASL_JAAS_CONFIG, "dynamicLoginModule required;")
|
||||
|
||||
val persistedProps = config.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true)
|
||||
assertFalse(persistedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG).contains("LoginModule"),
|
||||
assertFalse(persistedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG).contains("LoginModule"),
|
||||
"Password not encoded")
|
||||
config.dynamicConfig.updateBrokerConfig(0, persistedProps)
|
||||
assertEquals("dynamicLoginModule required;", config.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
assertEquals("dynamicLoginModule required;", config.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
|
||||
// New config with same secret should use the dynamic password config
|
||||
val newConfigWithSameSecret = KafkaConfig(props)
|
||||
newConfigWithSameSecret.dynamicConfig.initialize(None, None)
|
||||
newConfigWithSameSecret.dynamicConfig.updateBrokerConfig(0, persistedProps)
|
||||
assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
|
||||
// New config with new secret should use the dynamic password config if new and old secrets are configured in KafkaConfig
|
||||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "new-encoder-secret")
|
||||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_OLD_SECRET_CONFIG, "config-encoder-secret")
|
||||
val newConfigWithNewAndOldSecret = KafkaConfig(props)
|
||||
newConfigWithNewAndOldSecret.dynamicConfig.updateBrokerConfig(0, persistedProps)
|
||||
assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
|
||||
// New config with new secret alone should revert to static password config since dynamic config cannot be decoded
|
||||
props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "another-new-encoder-secret")
|
||||
val newConfigWithNewSecret = KafkaConfig(props)
|
||||
newConfigWithNewSecret.dynamicConfig.updateBrokerConfig(0, persistedProps)
|
||||
assertEquals("staticLoginModule required;", newConfigWithNewSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
assertEquals("staticLoginModule required;", newConfigWithNewSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -25,12 +25,13 @@ import kafka.security.authorizer.AclAuthorizer
|
|||
import kafka.utils.TestUtils.assertBadConfigContainingMessage
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import org.apache.kafka.common.Node
|
||||
import org.apache.kafka.common.config.{ConfigException, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigException, SaslConfigs, SecurityConfig, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.metrics.Sensor
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.record.{CompressionType, Records}
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
|
||||
import org.apache.kafka.coordinator.group.Group.GroupType
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
|
@ -40,7 +41,7 @@ import org.apache.kafka.raft.QuorumConfig
|
|||
import org.apache.kafka.security.PasswordEncoderConfigs
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs}
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.server.metrics.MetricConfigs
|
||||
import org.apache.kafka.storage.internals.log.CleanerConfig
|
||||
|
@ -948,65 +949,65 @@ class KafkaConfigTest {
|
|||
case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1)
|
||||
|
||||
//SSL Configs
|
||||
case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG =>
|
||||
case KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_PROVIDER_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG =>
|
||||
case KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG =>
|
||||
case KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string
|
||||
case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG =>
|
||||
case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG =>
|
||||
case SslConfigs.SSL_PROTOCOL_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_PROVIDER_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG =>
|
||||
case SslConfigs.SSL_KEYSTORE_TYPE_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEY_PASSWORD_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEYSTORE_KEY_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG =>
|
||||
case SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG =>
|
||||
case BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG => // ignore string
|
||||
case SslConfigs.SSL_CIPHER_SUITES_CONFIG => // ignore string
|
||||
case BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string
|
||||
|
||||
//Sasl Configs
|
||||
case KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG => // ignore
|
||||
case KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore
|
||||
case KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG => // ignore string
|
||||
case KafkaSecurityConfigs.SASL_JAAS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG =>
|
||||
case KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG =>
|
||||
case BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore
|
||||
case BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG =>
|
||||
case SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS =>
|
||||
case BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG =>
|
||||
case SaslConfigs.SASL_LOGIN_CLASS =>
|
||||
case SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS =>
|
||||
case SaslConfigs.SASL_KERBEROS_SERVICE_NAME => // ignore string
|
||||
case SaslConfigs.SASL_KERBEROS_KINIT_CMD =>
|
||||
case SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR =>
|
||||
case SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER =>
|
||||
case SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN =>
|
||||
case BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG => // ignore string
|
||||
case SaslConfigs.SASL_JAAS_CONFIG =>
|
||||
case SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR =>
|
||||
case SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER =>
|
||||
case SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS =>
|
||||
case SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS =>
|
||||
case SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS =>
|
||||
case SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS =>
|
||||
case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS =>
|
||||
case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE =>
|
||||
case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER =>
|
||||
|
||||
// Security config
|
||||
case KafkaSecurityConfigs.SECURITY_PROVIDER_CLASS_CONFIG =>
|
||||
case SecurityConfig.SECURITY_PROVIDERS_CONFIG =>
|
||||
|
||||
// Password encoder configs
|
||||
case PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG =>
|
||||
|
@ -1027,7 +1028,7 @@ class KafkaConfigTest {
|
|||
case MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG => // ignore
|
||||
case MetricConfigs.KAFKA_METRICS_POLLING_INTERVAL_SECONDS_CONFIG => //ignore
|
||||
|
||||
case KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||
case BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||
|
||||
// Raft Quorum Configs
|
||||
case QuorumConfig.QUORUM_VOTERS_CONFIG => // ignore string
|
||||
|
@ -1642,8 +1643,8 @@ class KafkaConfigTest {
|
|||
val props = new Properties()
|
||||
props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
|
||||
val config = KafkaConfig.fromProps(props)
|
||||
assertNotNull(config.getLong(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG))
|
||||
assertNotNull(config.getLong(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG))
|
||||
assertNotNull(config.getLong(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS))
|
||||
assertNotNull(config.getLong(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS))
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -20,6 +20,7 @@ import kafka.utils.TestUtils
|
|||
import org.apache.kafka.common._
|
||||
import org.apache.kafka.common.acl._
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
|
@ -45,7 +46,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
|||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||
import org.apache.kafka.network.Session
|
||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
||||
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs}
|
||||
import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -83,7 +84,7 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1")
|
||||
properties.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100")
|
||||
properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0")
|
||||
properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName)
|
||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName)
|
||||
properties.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true")
|
||||
if (isKRaftTest()) {
|
||||
properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.KraftTestAuthorizer].getName)
|
||||
|
@ -94,7 +95,7 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
|
||||
override def kraftControllerConfigs(): Seq[Properties] = {
|
||||
val properties = new Properties()
|
||||
properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName)
|
||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName)
|
||||
Seq(properties)
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
|||
import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse}
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.extension.ExtendWith
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||
|
@ -46,7 +45,7 @@ object SaslApiVersionsRequestTest {
|
|||
|
||||
def saslApiVersionsRequestClusterConfig(): java.util.List[ClusterConfig] = {
|
||||
val saslServerProperties = new java.util.HashMap[String, String]()
|
||||
saslServerProperties.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism)
|
||||
saslServerProperties.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism)
|
||||
saslServerProperties.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(","))
|
||||
|
||||
val saslClientProperties = new java.util.HashMap[String, String]()
|
||||
|
|
|
@ -18,11 +18,10 @@ package kafka.utils
|
|||
|
||||
import java.io.{BufferedWriter, File, FileWriter}
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.Seq
|
||||
import org.apache.kafka.clients.admin.ScramMechanism
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.utils.Java
|
||||
import org.apache.kafka.server.config.KafkaSecurityConfigs
|
||||
|
||||
object JaasTestUtils {
|
||||
|
||||
|
@ -161,8 +160,8 @@ object JaasTestUtils {
|
|||
val result = saslProperties.getOrElse(new Properties)
|
||||
// IBM Kerberos module doesn't support the serviceName JAAS property, hence it needs to be
|
||||
// passed as a Kafka property
|
||||
if (isIbmSecurity && !result.contains(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG))
|
||||
result.put(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, serviceName)
|
||||
if (isIbmSecurity && !result.contains(SaslConfigs.SASL_KERBEROS_SERVICE_NAME))
|
||||
result.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, serviceName)
|
||||
result
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import kafka.server.ZkAdminManager
|
|||
import kafka.zk.{AdminZkClient, ZkMigrationClient}
|
||||
import org.apache.kafka.clients.admin.ScramMechanism
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.metadata.ClientQuotaRecord
|
||||
import org.apache.kafka.common.metadata.ClientQuotaRecord.EntityData
|
||||
import org.apache.kafka.common.metadata.ConfigRecord
|
||||
|
@ -37,7 +37,7 @@ import org.apache.kafka.metadata.RecordTestUtils
|
|||
import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter
|
||||
import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, QuotaConfigs}
|
||||
import org.apache.kafka.server.config.{ConfigType, QuotaConfigs, ReplicationConfigs}
|
||||
import org.apache.kafka.server.util.MockRandom
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
@ -63,7 +63,7 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness {
|
|||
// Create some configs and persist in Zk.
|
||||
val props = new Properties()
|
||||
props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "1") // normal config
|
||||
props.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, encoder.encode(new Password(SECRET))) // sensitive config
|
||||
props.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, encoder.encode(new Password(SECRET))) // sensitive config
|
||||
zkClient.setOrCreateEntityConfigs(ConfigType.BROKER, "1", props)
|
||||
|
||||
val defaultProps = new Properties()
|
||||
|
@ -83,7 +83,7 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness {
|
|||
|
||||
assertTrue(props.containsKey(name))
|
||||
// If the config is sensitive, compare it to the decoded value.
|
||||
if (name == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
if (name == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
assertEquals(SECRET, value)
|
||||
} else {
|
||||
assertEquals(props.getProperty(name), value)
|
||||
|
@ -100,13 +100,13 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness {
|
|||
// persisted in Zookeeper is encrypted.
|
||||
val newProps = new util.HashMap[String, String]()
|
||||
newProps.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") // normal config
|
||||
newProps.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, NEW_SECRET) // sensitive config
|
||||
newProps.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, NEW_SECRET) // sensitive config
|
||||
migrationState = migrationClient.configClient().writeConfigs(
|
||||
new ConfigResource(ConfigResource.Type.BROKER, "1"), newProps, migrationState)
|
||||
val actualPropsInZk = zkClient.getEntityConfigs(ConfigType.BROKER, "1")
|
||||
assertEquals(2, actualPropsInZk.size())
|
||||
actualPropsInZk.forEach { case (key, value) =>
|
||||
if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
assertEquals(NEW_SECRET, encoder.decode(value.toString).value)
|
||||
} else {
|
||||
assertEquals(newProps.get(key), value)
|
||||
|
|
|
@ -19,7 +19,7 @@ package kafka.zk.migration
|
|||
import kafka.api.LeaderAndIsr
|
||||
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
|
||||
import kafka.coordinator.transaction.{ProducerIdManager, ZkProducerIdManager}
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig}
|
||||
import org.apache.kafka.common.errors.ControllerMovedException
|
||||
import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType, PartitionRecord, ProducerIdsRecord, TopicRecord}
|
||||
import org.apache.kafka.common.{DirectoryId, TopicPartition, Uuid}
|
||||
|
@ -28,7 +28,7 @@ import org.apache.kafka.metadata.migration.{KRaftMigrationZkWriter, ZkMigrationL
|
|||
import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration}
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs}
|
||||
import org.apache.kafka.server.config.ConfigType
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -331,7 +331,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
|
|||
val topicId = Uuid.randomUuid()
|
||||
val props = new Properties()
|
||||
props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "1") // normal config
|
||||
props.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SECRET) // sensitive config
|
||||
props.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SECRET) // sensitive config
|
||||
|
||||
// // Leave Zk in an incomplete state.
|
||||
// zkClient.createTopicAssignment(topicName, Some(topicId), Map(tp -> Seq(1)))
|
||||
|
@ -402,7 +402,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
|
|||
assertEquals(2, brokerProps.size())
|
||||
|
||||
brokerProps.asScala.foreach { case (key, value) =>
|
||||
if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
assertEquals(SECRET, encoder.decode(value).value)
|
||||
} else {
|
||||
assertEquals(props.getProperty(key), value)
|
||||
|
@ -410,7 +410,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
|
|||
}
|
||||
|
||||
topicProps.asScala.foreach { case (key, value) =>
|
||||
if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) {
|
||||
assertEquals(SECRET, encoder.decode(value).value)
|
||||
} else {
|
||||
assertEquals(props.getProperty(key), value)
|
||||
|
|
|
@ -1,239 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.server.config;
|
||||
|
||||
import org.apache.kafka.common.config.SaslConfigs;
|
||||
import org.apache.kafka.common.config.SecurityConfig;
|
||||
import org.apache.kafka.common.config.SslClientAuth;
|
||||
import org.apache.kafka.common.config.SslConfigs;
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder;
|
||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Common home for broker-side security configs which need to be accessible from the libraries shared
|
||||
* between the broker and the multiple modules in Kafka.
|
||||
*
|
||||
* Note this is an internal API and subject to change without notice.
|
||||
*/
|
||||
public class KafkaSecurityConfigs {
|
||||
|
||||
/** ********* SSL Configuration ****************/
|
||||
public final static String SSL_PROTOCOL_CONFIG = SslConfigs.SSL_PROTOCOL_CONFIG;
|
||||
public final static String SSL_PROTOCOL_DOC = SslConfigs.SSL_PROTOCOL_DOC;
|
||||
public static final String SSL_PROTOCOL_DEFAULT = SslConfigs.DEFAULT_SSL_PROTOCOL;
|
||||
|
||||
public final static String SSL_PROVIDER_CONFIG = SslConfigs.SSL_PROVIDER_CONFIG;
|
||||
public final static String SSL_PROVIDER_DOC = SslConfigs.SSL_PROVIDER_DOC;
|
||||
|
||||
public final static String SSL_CIPHER_SUITES_CONFIG = SslConfigs.SSL_CIPHER_SUITES_CONFIG;
|
||||
public final static String SSL_CIPHER_SUITES_DOC = SslConfigs.SSL_CIPHER_SUITES_DOC;
|
||||
|
||||
public final static String SSL_ENABLED_PROTOCOLS_CONFIG = SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG;
|
||||
public final static String SSL_ENABLED_PROTOCOLS_DOC = SslConfigs.SSL_ENABLED_PROTOCOLS_DOC;
|
||||
public static final String SSL_ENABLED_PROTOCOLS_DEFAULTS = SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS;
|
||||
|
||||
public final static String SSL_KEYSTORE_TYPE_CONFIG = SslConfigs.SSL_KEYSTORE_TYPE_CONFIG;
|
||||
public final static String SSL_KEYSTORE_TYPE_DOC = SslConfigs.SSL_KEYSTORE_TYPE_DOC;
|
||||
public static final String SSL_KEYSTORE_TYPE_DEFAULT = SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE;
|
||||
|
||||
public final static String SSL_KEYSTORE_LOCATION_CONFIG = SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG;
|
||||
public final static String SSL_KEYSTORE_LOCATION_DOC = SslConfigs.SSL_KEYSTORE_LOCATION_DOC;
|
||||
|
||||
public final static String SSL_KEYSTORE_PASSWORD_CONFIG = SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG;
|
||||
public final static String SSL_KEYSTORE_PASSWORD_DOC = SslConfigs.SSL_KEYSTORE_PASSWORD_DOC;
|
||||
|
||||
public final static String SSL_KEY_PASSWORD_CONFIG = SslConfigs.SSL_KEY_PASSWORD_CONFIG;
|
||||
public final static String SSL_KEY_PASSWORD_DOC = SslConfigs.SSL_KEY_PASSWORD_DOC;
|
||||
|
||||
public final static String SSL_KEYSTORE_KEY_CONFIG = SslConfigs.SSL_KEYSTORE_KEY_CONFIG;
|
||||
public final static String SSL_KEYSTORE_KEY_DOC = SslConfigs.SSL_KEYSTORE_KEY_DOC;
|
||||
|
||||
public final static String SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG = SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG;
|
||||
public final static String SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC = SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC;
|
||||
public final static String SSL_TRUSTSTORE_TYPE_CONFIG = SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG;
|
||||
public final static String SSL_TRUSTSTORE_TYPE_DOC = SslConfigs.SSL_TRUSTSTORE_TYPE_DOC;
|
||||
public static final String SSL_TRUSTSTORE_TYPE_DEFAULT = SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE;
|
||||
|
||||
public final static String SSL_TRUSTSTORE_LOCATION_CONFIG = SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
|
||||
public final static String SSL_TRUSTSTORE_PASSWORD_DOC = SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC;
|
||||
|
||||
public final static String SSL_TRUSTSTORE_PASSWORD_CONFIG = SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
|
||||
public final static String SSL_TRUSTSTORE_LOCATION_DOC = SslConfigs.SSL_TRUSTSTORE_LOCATION_DOC;
|
||||
|
||||
public final static String SSL_TRUSTSTORE_CERTIFICATES_CONFIG = SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG;
|
||||
public final static String SSL_TRUSTSTORE_CERTIFICATES_DOC = SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC;
|
||||
|
||||
public final static String SSL_KEYMANAGER_ALGORITHM_CONFIG = SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG;
|
||||
public final static String SSL_KEYMANAGER_ALGORITHM_DOC = SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC;
|
||||
public static final String SSL_KEYMANAGER_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM;
|
||||
|
||||
public final static String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG;
|
||||
public final static String SSL_TRUSTMANAGER_ALGORITHM_DOC = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC;
|
||||
public static final String SSL_TRUSTMANAGER_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM;
|
||||
|
||||
public final static String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG;
|
||||
public final static String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC;
|
||||
public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM;
|
||||
|
||||
public final static String SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG;
|
||||
public final static String SSL_SECURE_RANDOM_IMPLEMENTATION_DOC = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC;
|
||||
|
||||
public final static String SSL_CLIENT_AUTH_CONFIG = BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG;
|
||||
public final static String SSL_CLIENT_AUTH_DOC = BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC;
|
||||
public static final String SSL_CLIENT_AUTH_DEFAULT = SslClientAuth.NONE.name().toLowerCase(Locale.ROOT);
|
||||
public static final String[] SSL_CLIENT_AUTHENTICATION_VALID_VALUES = SslClientAuth.VALUES.stream()
|
||||
.map(v -> v.toString().toLowerCase(Locale.ROOT)).toArray(String[]::new);
|
||||
|
||||
public final static String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG;
|
||||
public final static String SSL_PRINCIPAL_MAPPING_RULES_DOC = BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC;
|
||||
public static final String SSL_PRINCIPAL_MAPPING_RULES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES;
|
||||
|
||||
public final static String SSL_ENGINE_FACTORY_CLASS_CONFIG = SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG;
|
||||
public final static String SSL_ENGINE_FACTORY_CLASS_DOC = SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC;
|
||||
|
||||
public final static String SSL_ALLOW_DN_CHANGES_CONFIG = BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG;
|
||||
public final static String SSL_ALLOW_DN_CHANGES_DOC = BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC;
|
||||
public final static boolean SSL_ALLOW_DN_CHANGES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE;
|
||||
|
||||
public final static String SSL_ALLOW_SAN_CHANGES_CONFIG = BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG;
|
||||
public final static String SSL_ALLOW_SAN_CHANGES_DOC = BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC;
|
||||
public final static boolean SSL_ALLOW_SAN_CHANGES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE;
|
||||
|
||||
/** ********* SASL Configuration ****************/
|
||||
public final static String SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG = BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG;
|
||||
public final static String SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC = BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC;
|
||||
|
||||
public static final String SASL_MECHANISM_INTER_BROKER_PROTOCOL_DEFAULT = SaslConfigs.DEFAULT_SASL_MECHANISM;
|
||||
|
||||
public final static String SASL_JAAS_CONFIG = SaslConfigs.SASL_JAAS_CONFIG;
|
||||
public final static String SASL_JAAS_CONFIG_DOC = SaslConfigs.SASL_JAAS_CONFIG_DOC;
|
||||
public final static String SASL_ENABLED_MECHANISMS_CONFIG = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG;
|
||||
public final static String SASL_ENABLED_MECHANISMS_DOC = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC;
|
||||
public static final List<String> SASL_ENABLED_MECHANISMS_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS;
|
||||
|
||||
public final static String SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG = BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS;
|
||||
public final static String SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC = BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC;
|
||||
public final static String SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG = SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS;
|
||||
public final static String SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC = SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC;
|
||||
public final static String SASL_LOGIN_CLASS_CONFIG = SaslConfigs.SASL_LOGIN_CLASS;
|
||||
public final static String SASL_LOGIN_CLASS_DOC = SaslConfigs.SASL_LOGIN_CLASS_DOC;
|
||||
public final static String SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG = SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS;
|
||||
public final static String SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC = SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC;
|
||||
public final static String SASL_KERBEROS_SERVICE_NAME_CONFIG = SaslConfigs.SASL_KERBEROS_SERVICE_NAME;
|
||||
public final static String SASL_KERBEROS_SERVICE_NAME_DOC = SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC;
|
||||
public final static String SASL_KERBEROS_KINIT_CMD_CONFIG = SaslConfigs.SASL_KERBEROS_KINIT_CMD;
|
||||
public final static String SASL_KERBEROS_KINIT_CMD_DOC = SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC;
|
||||
public static final String SASL_KERBEROS_KINIT_CMD_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD;
|
||||
|
||||
public final static String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR;
|
||||
public final static String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC;
|
||||
public static final double SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR;
|
||||
|
||||
public final static String SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER;
|
||||
public final static String SASL_KERBEROS_TICKET_RENEW_JITTER_DOC = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC;
|
||||
public static final double SASL_KERBEROS_TICKET_RENEW_JITTER_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER;
|
||||
|
||||
public final static String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN;
|
||||
public final static String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC;
|
||||
public static final long SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN;
|
||||
|
||||
public final static String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG;
|
||||
public final static String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC;
|
||||
public static final List<String> SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES;
|
||||
|
||||
public final static String SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR;
|
||||
public final static String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC;
|
||||
public static final double SASL_LOGIN_REFRESH_WINDOW_FACTOR_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR;
|
||||
|
||||
public final static String SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER;
|
||||
public final static String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC;
|
||||
public static final double SASL_LOGIN_REFRESH_WINDOW_JITTER_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER;
|
||||
|
||||
public final static String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS;
|
||||
public final static String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC;
|
||||
public static final short SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS;
|
||||
|
||||
public final static String SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS;
|
||||
public final static String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC;
|
||||
public static final short SASL_LOGIN_REFRESH_BUFFER_SECONDS_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS;
|
||||
|
||||
public final static String SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG = SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
|
||||
public final static String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC;
|
||||
public final static String SASL_LOGIN_READ_TIMEOUT_MS_CONFIG = SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
|
||||
public final static String SASL_LOGIN_READ_TIMEOUT_MS_DOC = SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC;
|
||||
public final static String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
|
||||
public final static String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC;
|
||||
public static final long SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
|
||||
|
||||
public final static String SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
|
||||
public final static String SASL_LOGIN_RETRY_BACKOFF_MS_DOC = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC;
|
||||
public static final long SASL_LOGIN_RETRY_BACKOFF_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG = SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
|
||||
public final static String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC;
|
||||
public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG = SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
|
||||
public final static String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC = SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC;
|
||||
public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG = SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
|
||||
public final static String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC = SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC;
|
||||
public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC;
|
||||
public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
|
||||
public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC;
|
||||
public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
|
||||
public final static String SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC = SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC;
|
||||
public static final int SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
|
||||
|
||||
public final static String SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
|
||||
public final static String SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC;
|
||||
public final static String SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
|
||||
public final static String SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC;
|
||||
|
||||
/** ******** Common Security Configuration *************/
|
||||
public final static String PRINCIPAL_BUILDER_CLASS_CONFIG = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG;
|
||||
public final static String PRINCIPAL_BUILDER_CLASS_DOC = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC;
|
||||
public static final Class<? extends KafkaPrincipalBuilder> PRINCIPAL_BUILDER_CLASS_DEFAULT = DefaultKafkaPrincipalBuilder.class;
|
||||
|
||||
public final static String CONNECTIONS_MAX_REAUTH_MS_CONFIG = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS;
|
||||
public final static String CONNECTIONS_MAX_REAUTH_MS_DOC = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC;
|
||||
public static final long CONNECTIONS_MAX_REAUTH_MS_DEFAULT = BrokerSecurityConfigs.DEFAULT_CONNECTIONS_MAX_REAUTH_MS;
|
||||
|
||||
public final static String SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG;
|
||||
public final static String SASL_SERVER_MAX_RECEIVE_SIZE_DOC = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC;
|
||||
public static final int SASL_SERVER_MAX_RECEIVE_SIZE_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE;
|
||||
|
||||
public final static String SECURITY_PROVIDER_CLASS_CONFIG = SecurityConfig.SECURITY_PROVIDERS_CONFIG;
|
||||
public final static String SECURITY_PROVIDERS_DOC = SecurityConfig.SECURITY_PROVIDERS_DOC;
|
||||
}
|
Loading…
Reference in New Issue