You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2021/03/31 05:04:11 UTC
[kafka] branch trunk updated: KAFKA-12578: Remove deprecated
security classes/methods for 3.0 (#10435)
This is an automated email from the ASF dual-hosted git repository.
ijuma pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new e8754ba KAFKA-12578: Remove deprecated security classes/methods for 3.0 (#10435)
e8754ba is described below
commit e8754ba7a01f33ba503ef79de66a5f644cc2ced8
Author: Ismael Juma <is...@juma.me.uk>
AuthorDate: Tue Mar 30 22:02:16 2021 -0700
KAFKA-12578: Remove deprecated security classes/methods for 3.0 (#10435)
More specifically, remove deprecated:
- Constants in SslConfigs
- Constants in SaslConfigs
- AclBinding constructor
- AclBindingFilter constructor
- PrincipalBuilder and DefaultPrincipalBuilder classes
- ResourceFilter
Also simplify tests and code that no longer have to handle the removed `PrincipalBuilder`.
These removals seem non controversial. There is a straightforward alternative. The
deprecations happened in 1.0.0 and 2.0.0.
Reviewers: Chia-Ping Tsai <ch...@gmail.com>
---
.../org/apache/kafka/common/acl/AclBinding.java | 14 ---
.../apache/kafka/common/acl/AclBindingFilter.java | 14 ---
.../apache/kafka/common/config/SaslConfigs.java | 35 ------
.../org/apache/kafka/common/config/SslConfigs.java | 30 ------
.../config/internals/BrokerSecurityConfigs.java | 7 +-
.../kafka/common/network/ChannelBuilders.java | 24 +----
.../common/network/PlaintextChannelBuilder.java | 2 +-
.../kafka/common/network/SslChannelBuilder.java | 2 +-
.../org/apache/kafka/common/resource/Resource.java | 7 --
.../kafka/common/resource/ResourceFilter.java | 118 ---------------------
.../security/auth/DefaultPrincipalBuilder.java | 45 --------
.../common/security/auth/PrincipalBuilder.java | 52 ---------
.../DefaultKafkaPrincipalBuilder.java | 70 +-----------
.../authenticator/SaslServerAuthenticator.java | 2 +-
.../kafka/common/network/ChannelBuildersTest.java | 52 +--------
.../kafka/common/resource/ResourceFilterTest.java | 82 --------------
.../auth/DefaultKafkaPrincipalBuilderTest.java | 67 +-----------
docs/upgrade.html | 3 +
18 files changed, 18 insertions(+), 608 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
index 67dbfc0..c323426 100644
--- a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
+++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
@@ -18,8 +18,6 @@
package org.apache.kafka.common.acl;
import org.apache.kafka.common.annotation.InterfaceStability;
-import org.apache.kafka.common.resource.PatternType;
-import org.apache.kafka.common.resource.Resource;
import org.apache.kafka.common.resource.ResourcePattern;
import java.util.Objects;
@@ -46,18 +44,6 @@ public class AclBinding {
}
/**
- * Create an instance of this class with the provided parameters.
- *
- * @param resource non-null resource
- * @param entry non-null entry
- * @deprecated Since 2.0. Use {@link #AclBinding(ResourcePattern, AccessControlEntry)}
- */
- @Deprecated
- public AclBinding(Resource resource, AccessControlEntry entry) {
- this(new ResourcePattern(resource.resourceType(), resource.name(), PatternType.LITERAL), entry);
- }
-
- /**
* @return true if this binding has any UNKNOWN components.
*/
public boolean isUnknown() {
diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
index 3168ec6..7682386 100644
--- a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
+++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
@@ -18,8 +18,6 @@
package org.apache.kafka.common.acl;
import org.apache.kafka.common.annotation.InterfaceStability;
-import org.apache.kafka.common.resource.PatternType;
-import org.apache.kafka.common.resource.ResourceFilter;
import org.apache.kafka.common.resource.ResourcePatternFilter;
import java.util.Objects;
@@ -51,18 +49,6 @@ public class AclBindingFilter {
}
/**
- * Create an instance of this filter with the provided parameters.
- *
- * @param resourceFilter non-null resource filter
- * @param entryFilter non-null access control entry filter
- * @deprecated Since 2.0. Use {@link #AclBindingFilter(ResourcePatternFilter, AccessControlEntryFilter)}
- */
- @Deprecated
- public AclBindingFilter(ResourceFilter resourceFilter, AccessControlEntryFilter entryFilter) {
- this(new ResourcePatternFilter(resourceFilter.resourceType(), resourceFilter.name(), PatternType.LITERAL), entryFilter);
- }
-
- /**
* @return {@code true} if this filter has any UNKNOWN components.
*/
public boolean isUnknown() {
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
index c0c3e5b..26897b2 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
@@ -17,9 +17,6 @@
package org.apache.kafka.common.config;
import org.apache.kafka.common.config.ConfigDef.Range;
-import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
-
-import java.util.List;
public class SaslConfigs {
/*
@@ -31,22 +28,6 @@ public class SaslConfigs {
public static final String GSSAPI_MECHANISM = "GSSAPI";
public static final String DEFAULT_SASL_MECHANISM = GSSAPI_MECHANISM;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SASL_ENABLED_MECHANISMS = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SASL_ENABLED_MECHANISMS_DOC = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final List<String> DEFAULT_SASL_ENABLED_MECHANISMS = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS;
-
public static final String SASL_JAAS_CONFIG = "sasl.jaas.config";
public static final String SASL_JAAS_CONFIG_DOC = "JAAS login context parameters for SASL connections in the format used by JAAS configuration files. "
+ "JAAS configuration file format is described <a href=\"http://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/LoginConfigFile.html\">here</a>. "
@@ -118,22 +99,6 @@ public class SaslConfigs {
+ " Currently applies only to OAUTHBEARER.";
public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final List<String> DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES;
-
public static void addClientSaslSupport(ConfigDef config) {
config.define(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC)
.define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, ConfigDef.Type.STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
index 55a58ac..d7ed803 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
@@ -30,25 +30,6 @@ public class SslConfigs {
* NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
*/
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String PRINCIPAL_BUILDER_CLASS_CONFIG = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String PRINCIPAL_BUILDER_CLASS_DOC = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release. In recent versions,
- * the config is optional and there is no default.
- */
- // use FQN to avoid import deprecation warning
- @Deprecated
- public static final String DEFAULT_PRINCIPAL_BUILDER_CLASS =
- org.apache.kafka.common.security.auth.DefaultPrincipalBuilder.class.getName();
-
public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol";
public static final String SSL_PROTOCOL_DOC = "The SSL protocol used to generate the SSLContext. "
+ "The default is 'TLSv1.3' when running with Java 11 or newer, 'TLSv1.2' otherwise. "
@@ -149,17 +130,6 @@ public class SslConfigs {
public static final String SSL_ENGINE_FACTORY_CLASS_CONFIG = "ssl.engine.factory.class";
public static final String SSL_ENGINE_FACTORY_CLASS_DOC = "The class of type org.apache.kafka.common.security.auth.SslEngineFactory to provide SSLEngine objects. Default value is org.apache.kafka.common.security.ssl.DefaultSslEngineFactory";
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SSL_CLIENT_AUTH_CONFIG = BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG;
- /**
- * @deprecated As of 1.0.0. This field will be removed in a future major release.
- */
- @Deprecated
- public static final String SSL_CLIENT_AUTH_DOC = BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC;
-
public static void addClientSslSupport(ConfigDef config) {
config.define(SslConfigs.SSL_PROTOCOL_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROTOCOL_DOC)
.define(SslConfigs.SSL_PROVIDER_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROVIDER_DOC)
diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java
index 3b84908..0b90da8 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java
@@ -39,10 +39,9 @@ public class BrokerSecurityConfigs {
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. This config also supports the deprecated PrincipalBuilder interface which was previously " +
- "used for client authentication over SSL. If no principal builder is defined, the default behavior depends " +
- "on the security protocol in use. For SSL authentication, the principal will be derived using the" +
- " rules defined by <code>" + SSL_PRINCIPAL_MAPPING_RULES_CONFIG + "</code> applied on the distinguished " +
+ "authorization. If no principal builder is defined, the default behavior depends " +
+ "on the security protocol in use. For SSL authentication, the principal will be derived using the " +
+ "rules defined by <code>" + SSL_PRINCIPAL_MAPPING_RULES_CONFIG + "</code> applied on the distinguished " +
"name from the client certificate if one is provided; otherwise, if client authentication is not required, " +
"the principal name will be ANONYMOUS. For SASL authentication, the principal will be derived using the " +
"rules defined by <code>" + SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG + "</code> if GSSAPI is in use, " +
diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
index b4a1ce8..b2760ae 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
@@ -220,23 +220,7 @@ public class ChannelBuilders {
throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `" + securityProtocol + "`");
}
- // Use FQN to avoid deprecated import warnings
- @SuppressWarnings("deprecation")
- private static org.apache.kafka.common.security.auth.PrincipalBuilder createPrincipalBuilder(
- Class<?> principalBuilderClass, Map<String, ?> configs) {
- org.apache.kafka.common.security.auth.PrincipalBuilder principalBuilder;
- if (principalBuilderClass == null)
- principalBuilder = new org.apache.kafka.common.security.auth.DefaultPrincipalBuilder();
- else
- principalBuilder = (org.apache.kafka.common.security.auth.PrincipalBuilder) Utils.newInstance(principalBuilderClass);
- principalBuilder.configure(configs);
- return principalBuilder;
- }
-
- @SuppressWarnings("deprecation")
public static KafkaPrincipalBuilder createPrincipalBuilder(Map<String, ?> configs,
- TransportLayer transportLayer,
- Authenticator authenticator,
KerberosShortNamer kerberosShortNamer,
SslPrincipalMapper sslPrincipalMapper) {
Class<?> principalBuilderClass = (Class<?>) configs.get(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG);
@@ -246,15 +230,9 @@ public class ChannelBuilders {
builder = new DefaultKafkaPrincipalBuilder(kerberosShortNamer, sslPrincipalMapper);
} else if (KafkaPrincipalBuilder.class.isAssignableFrom(principalBuilderClass)) {
builder = (KafkaPrincipalBuilder) Utils.newInstance(principalBuilderClass);
- } else if (org.apache.kafka.common.security.auth.PrincipalBuilder.class.isAssignableFrom(principalBuilderClass)) {
- org.apache.kafka.common.security.auth.PrincipalBuilder oldPrincipalBuilder =
- createPrincipalBuilder(principalBuilderClass, configs);
- builder = DefaultKafkaPrincipalBuilder.fromOldPrincipalBuilder(authenticator, transportLayer,
- oldPrincipalBuilder, kerberosShortNamer);
} else {
throw new InvalidConfigurationException("Type " + principalBuilderClass.getName() + " is not " +
- "an instance of " + org.apache.kafka.common.security.auth.PrincipalBuilder.class.getName() + " or " +
- KafkaPrincipalBuilder.class.getName());
+ "an instance of " + KafkaPrincipalBuilder.class.getName());
}
if (builder instanceof Configurable)
diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
index 9369253..50bbc48 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
@@ -85,7 +85,7 @@ public class PlaintextChannelBuilder implements ChannelBuilder {
private PlaintextAuthenticator(Map<String, ?> configs, PlaintextTransportLayer transportLayer, ListenerName listenerName) {
this.transportLayer = transportLayer;
- this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, transportLayer, this, null, null);
+ this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, null, null);
this.listenerName = listenerName;
}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java
index 1140ea7..4dabf0a 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java
@@ -135,7 +135,7 @@ public class SslChannelBuilder implements ChannelBuilder, ListenerReconfigurable
private SslAuthenticator(Map<String, ?> configs, SslTransportLayer transportLayer, ListenerName listenerName, SslPrincipalMapper sslPrincipalMapper) {
this.transportLayer = transportLayer;
- this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, transportLayer, this, null, sslPrincipalMapper);
+ this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, null, sslPrincipalMapper);
this.listenerName = listenerName;
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/resource/Resource.java b/clients/src/main/java/org/apache/kafka/common/resource/Resource.java
index f41f41a..ebc5b8e 100644
--- a/clients/src/main/java/org/apache/kafka/common/resource/Resource.java
+++ b/clients/src/main/java/org/apache/kafka/common/resource/Resource.java
@@ -68,13 +68,6 @@ public class Resource {
return name;
}
- /**
- * Create a filter which matches only this Resource.
- */
- public ResourceFilter toFilter() {
- return new ResourceFilter(resourceType, name);
- }
-
@Override
public String toString() {
return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "<any>" : name) + ")";
diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java
deleted file mode 100644
index 2ea9032..0000000
--- a/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java
+++ /dev/null
@@ -1,118 +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.common.resource;
-
-import org.apache.kafka.common.annotation.InterfaceStability;
-
-import java.util.Objects;
-
-/**
- * A filter which matches Resource objects.
- *
- * The API for this class is still evolving and we may break compatibility in minor releases, if necessary.
- */
-@InterfaceStability.Evolving
-public class ResourceFilter {
- private final ResourceType resourceType;
- private final String name;
-
- /**
- * Matches any resource.
- */
- public static final ResourceFilter ANY = new ResourceFilter(ResourceType.ANY, null);
-
- /**
- * Create an instance of this class with the provided parameters.
- *
- * @param resourceType non-null resource type
- * @param name resource name or null
- */
- public ResourceFilter(ResourceType resourceType, String name) {
- Objects.requireNonNull(resourceType);
- this.resourceType = resourceType;
- this.name = name;
- }
-
- /**
- * Return the resource type.
- */
- public ResourceType resourceType() {
- return resourceType;
- }
-
- /**
- * Return the resource name or null.
- */
- public String name() {
- return name;
- }
-
- @Override
- public String toString() {
- return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "<any>" : name) + ")";
- }
-
- /**
- * Return true if this ResourceFilter has any UNKNOWN components.
- */
- public boolean isUnknown() {
- return resourceType.isUnknown();
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof ResourceFilter))
- return false;
- ResourceFilter other = (ResourceFilter) o;
- return resourceType.equals(other.resourceType) && Objects.equals(name, other.name);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(resourceType, name);
- }
-
- /**
- * Return true if this filter matches the given Resource.
- */
- public boolean matches(Resource other) {
- if ((name != null) && (!name.equals(other.name())))
- return false;
- return (resourceType == ResourceType.ANY) || (resourceType.equals(other.resourceType()));
- }
-
- /**
- * Return true if this filter could only match one ACE. In other words, if there are no ANY or UNKNOWN fields.
- */
- public boolean matchesAtMostOne() {
- return findIndefiniteField() == null;
- }
-
- /**
- * Return a string describing an ANY or UNKNOWN field, or null if there is no such field.
- */
- public String findIndefiniteField() {
- if (resourceType == ResourceType.ANY)
- return "Resource type is ANY.";
- if (resourceType == ResourceType.UNKNOWN)
- return "Resource type is UNKNOWN.";
- if (name == null)
- return "Resource name is NULL.";
- return null;
- }
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java
deleted file mode 100644
index 424cfaa..0000000
--- a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java
+++ /dev/null
@@ -1,45 +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.common.security.auth;
-
-import java.util.Map;
-import java.security.Principal;
-
-import org.apache.kafka.common.network.TransportLayer;
-import org.apache.kafka.common.network.Authenticator;
-import org.apache.kafka.common.KafkaException;
-
-/**
- * DefaultPrincipalBuilder which return transportLayer's peer Principal
- * @deprecated As of Kafka 1.0.0. This will be removed in a future major release.
- **/
-@Deprecated
-public class DefaultPrincipalBuilder implements PrincipalBuilder {
-
- public void configure(Map<String, ?> configs) {}
-
- public Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException {
- try {
- return transportLayer.peerPrincipal();
- } catch (Exception e) {
- throw new KafkaException("Failed to build principal due to: ", e);
- }
- }
-
- public void close() throws KafkaException {}
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java
deleted file mode 100644
index d58170a..0000000
--- a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java
+++ /dev/null
@@ -1,52 +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.common.security.auth;
-
-import org.apache.kafka.common.annotation.InterfaceStability;
-import org.apache.kafka.common.network.TransportLayer;
-import org.apache.kafka.common.network.Authenticator;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.Configurable;
-
-import java.util.Map;
-import java.security.Principal;
-
-/**
- * PrincipalBuilder for Authenticator
- * @deprecated As of Kafka 1.0.0, use {@link KafkaPrincipalBuilder} instead. This will be removed in
- * a future major release.
- */
-@InterfaceStability.Unstable
-@Deprecated
-public interface PrincipalBuilder extends Configurable {
-
- /**
- * Configures this class with given key-value pairs.
- */
- void configure(Map<String, ?> configs);
-
- /**
- * Returns Principal.
- */
- Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException;
-
- /**
- * Closes this instance.
- */
- void close() throws KafkaException;
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultKafkaPrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultKafkaPrincipalBuilder.java
index 384f967..cae0796 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultKafkaPrincipalBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultKafkaPrincipalBuilder.java
@@ -22,8 +22,6 @@ import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.message.DefaultPrincipalData;
-import org.apache.kafka.common.network.Authenticator;
-import org.apache.kafka.common.network.TransportLayer;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.security.auth.AuthenticationContext;
@@ -41,89 +39,39 @@ import javax.net.ssl.SSLSession;
import javax.security.sasl.SaslServer;
import org.apache.kafka.common.security.ssl.SslPrincipalMapper;
-import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.security.Principal;
-import static java.util.Objects.requireNonNull;
-
/**
* Default implementation of {@link KafkaPrincipalBuilder} which provides basic support for
* SSL authentication and SASL authentication. In the latter case, when GSSAPI is used, this
* class applies {@link org.apache.kafka.common.security.kerberos.KerberosShortNamer} to transform
* the name.
*
- * NOTE: This is an internal class and can change without notice. Unlike normal implementations
- * of {@link KafkaPrincipalBuilder}, there is no default no-arg constructor since this class
- * must adapt implementations of the older {@link org.apache.kafka.common.security.auth.PrincipalBuilder} interface.
+ * NOTE: This is an internal class and can change without notice.
*/
-public class DefaultKafkaPrincipalBuilder implements KafkaPrincipalBuilder, KafkaPrincipalSerde, Closeable {
- // Use FQN to avoid import deprecation warnings
- @SuppressWarnings("deprecation")
- private final org.apache.kafka.common.security.auth.PrincipalBuilder oldPrincipalBuilder;
- private final Authenticator authenticator;
- private final TransportLayer transportLayer;
+public class DefaultKafkaPrincipalBuilder implements KafkaPrincipalBuilder, KafkaPrincipalSerde {
private final KerberosShortNamer kerberosShortNamer;
private final SslPrincipalMapper sslPrincipalMapper;
/**
- * Construct a new instance which wraps an instance of the older {@link org.apache.kafka.common.security.auth.PrincipalBuilder}.
- *
- * @param authenticator The authenticator in use
- * @param transportLayer The underlying transport layer
- * @param oldPrincipalBuilder Instance of {@link org.apache.kafka.common.security.auth.PrincipalBuilder}
- * @param kerberosShortNamer Kerberos name rewrite rules or null if none have been configured
- */
- @SuppressWarnings("deprecation")
- public static DefaultKafkaPrincipalBuilder fromOldPrincipalBuilder(Authenticator authenticator,
- TransportLayer transportLayer,
- org.apache.kafka.common.security.auth.PrincipalBuilder oldPrincipalBuilder,
- KerberosShortNamer kerberosShortNamer) {
- return new DefaultKafkaPrincipalBuilder(
- requireNonNull(authenticator),
- requireNonNull(transportLayer),
- requireNonNull(oldPrincipalBuilder),
- kerberosShortNamer,
- null);
- }
-
- @SuppressWarnings("deprecation")
- private DefaultKafkaPrincipalBuilder(Authenticator authenticator,
- TransportLayer transportLayer,
- org.apache.kafka.common.security.auth.PrincipalBuilder oldPrincipalBuilder,
- KerberosShortNamer kerberosShortNamer,
- SslPrincipalMapper sslPrincipalMapper) {
- this.authenticator = authenticator;
- this.transportLayer = transportLayer;
- this.oldPrincipalBuilder = oldPrincipalBuilder;
- this.kerberosShortNamer = kerberosShortNamer;
- this.sslPrincipalMapper = sslPrincipalMapper;
- }
-
- /**
* Construct a new instance.
*
* @param kerberosShortNamer Kerberos name rewrite rules or null if none have been configured
* @param sslPrincipalMapper SSL Principal mapper or null if none have been configured
*/
public DefaultKafkaPrincipalBuilder(KerberosShortNamer kerberosShortNamer, SslPrincipalMapper sslPrincipalMapper) {
- this(null, null, null, kerberosShortNamer, sslPrincipalMapper);
+ this.kerberosShortNamer = kerberosShortNamer;
+ this.sslPrincipalMapper = sslPrincipalMapper;
}
@Override
public KafkaPrincipal build(AuthenticationContext context) {
if (context instanceof PlaintextAuthenticationContext) {
- if (oldPrincipalBuilder != null)
- return convertToKafkaPrincipal(oldPrincipalBuilder.buildPrincipal(transportLayer, authenticator));
-
return KafkaPrincipal.ANONYMOUS;
} else if (context instanceof SslAuthenticationContext) {
SSLSession sslSession = ((SslAuthenticationContext) context).session();
-
- if (oldPrincipalBuilder != null)
- return convertToKafkaPrincipal(oldPrincipalBuilder.buildPrincipal(transportLayer, authenticator));
-
try {
return applySslPrincipalMapper(sslSession.getPeerPrincipal());
} catch (SSLPeerUnverifiedException se) {
@@ -164,10 +112,6 @@ public class DefaultKafkaPrincipalBuilder implements KafkaPrincipalBuilder, Kafk
}
}
- private KafkaPrincipal convertToKafkaPrincipal(Principal principal) {
- return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, principal.getName());
- }
-
@Override
public byte[] serialize(KafkaPrincipal principal) {
DefaultPrincipalData data = new DefaultPrincipalData()
@@ -188,10 +132,4 @@ public class DefaultKafkaPrincipalBuilder implements KafkaPrincipalBuilder, Kafk
DefaultPrincipalData data = new DefaultPrincipalData(new ByteBufferAccessor(buffer), version);
return new KafkaPrincipal(data.type(), data.name(), data.tokenAuthenticated());
}
-
- @Override
- public void close() {
- if (oldPrincipalBuilder != null)
- oldPrincipalBuilder.close();
- }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
index e122c18..6e35ee7 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
@@ -188,7 +188,7 @@ public class SaslServerAuthenticator implements Authenticator {
// Note that the old principal builder does not support SASL, so we do not need to pass the
// authenticator or the transport layer
- this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, null, null, kerberosNameParser, null);
+ this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, kerberosNameParser, null);
}
private void createSaslServer(String mechanism) throws IOException {
diff --git a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java
index f58b26d..f1d367b 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java
@@ -17,18 +17,13 @@
package org.apache.kafka.common.network;
import org.apache.kafka.common.Configurable;
-import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
import org.apache.kafka.common.security.TestSecurityConfig;
import org.apache.kafka.common.security.auth.AuthenticationContext;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder;
-import org.apache.kafka.common.security.auth.PlaintextAuthenticationContext;
-import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.junit.jupiter.api.Test;
-import java.net.InetAddress;
-import java.security.Principal;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
@@ -37,33 +32,14 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.mock;
public class ChannelBuildersTest {
@Test
- public void testCreateOldPrincipalBuilder() throws Exception {
- TransportLayer transportLayer = mock(TransportLayer.class);
- Authenticator authenticator = mock(Authenticator.class);
-
- Map<String, Object> configs = new HashMap<>();
- configs.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, OldPrincipalBuilder.class);
- KafkaPrincipalBuilder builder = ChannelBuilders.createPrincipalBuilder(configs, transportLayer, authenticator, null, null);
-
- // test old principal builder is properly configured and delegated to
- assertTrue(OldPrincipalBuilder.configured);
-
- // test delegation
- KafkaPrincipal principal = builder.build(new PlaintextAuthenticationContext(InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name()));
- assertEquals(OldPrincipalBuilder.PRINCIPAL_NAME, principal.getName());
- assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
- }
-
- @Test
public void testCreateConfigurableKafkaPrincipalBuilder() {
Map<String, Object> configs = new HashMap<>();
configs.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, ConfigurableKafkaPrincipalBuilder.class);
- KafkaPrincipalBuilder builder = ChannelBuilders.createPrincipalBuilder(configs, null, null, null, null);
+ KafkaPrincipalBuilder builder = ChannelBuilders.createPrincipalBuilder(configs, null, null);
assertTrue(builder instanceof ConfigurableKafkaPrincipalBuilder);
assertTrue(((ConfigurableKafkaPrincipalBuilder) builder).configured);
}
@@ -128,32 +104,6 @@ public class ChannelBuildersTest {
assertFalse(securityConfig.unused().contains("custom.config2.key"));
}
- @SuppressWarnings("deprecation")
- public static class OldPrincipalBuilder implements org.apache.kafka.common.security.auth.PrincipalBuilder {
- private static boolean configured = false;
- private static final String PRINCIPAL_NAME = "bob";
-
- @Override
- public void configure(Map<String, ?> configs) {
- configured = true;
- }
-
- @Override
- public Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException {
- return new Principal() {
- @Override
- public String getName() {
- return PRINCIPAL_NAME;
- }
- };
- }
-
- @Override
- public void close() throws KafkaException {
-
- }
- }
-
public static class ConfigurableKafkaPrincipalBuilder implements KafkaPrincipalBuilder, Configurable {
private boolean configured = false;
diff --git a/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java b/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java
deleted file mode 100644
index 4ff3c70..0000000
--- a/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java
+++ /dev/null
@@ -1,82 +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.common.resource;
-
-import org.junit.jupiter.api.Test;
-
-import static org.apache.kafka.common.resource.ResourceType.ANY;
-import static org.apache.kafka.common.resource.ResourceType.GROUP;
-import static org.apache.kafka.common.resource.ResourceType.TOPIC;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class ResourceFilterTest {
- @Test
- public void shouldNotMatchIfDifferentResourceType() {
- assertFalse(new ResourceFilter(TOPIC, "Name")
- .matches(new Resource(GROUP, "Name")));
- }
-
- @Test
- public void shouldNotMatchIfDifferentName() {
- assertFalse(new ResourceFilter(TOPIC, "Different")
- .matches(new Resource(TOPIC, "Name")));
- }
-
- @Test
- public void shouldNotMatchIfDifferentNameCase() {
- assertFalse(new ResourceFilter(TOPIC, "NAME")
- .matches(new Resource(TOPIC, "Name")));
- }
-
- @Test
- public void shouldMatchWhereResourceTypeIsAny() {
- assertTrue(new ResourceFilter(ANY, "Name")
- .matches(new Resource(TOPIC, "Name")));
- }
-
- @Test
- public void shouldMatchWhereResourceNameIsAny() {
- assertTrue(new ResourceFilter(TOPIC, null)
- .matches(new Resource(TOPIC, "Name")));
- }
-
- @Test
- public void shouldMatchIfExactMatch() {
- assertTrue(new ResourceFilter(TOPIC, "Name")
- .matches(new Resource(TOPIC, "Name")));
- }
-
- @Test
- public void shouldMatchWildcardIfExactMatch() {
- assertTrue(new ResourceFilter(TOPIC, "*")
- .matches(new Resource(TOPIC, "*")));
- }
-
- @Test
- public void shouldNotMatchWildcardAgainstOtherName() {
- assertFalse(new ResourceFilter(TOPIC, "Name")
- .matches(new Resource(TOPIC, "*")));
- }
-
- @Test
- public void shouldNotMatchLiteralWildcardTheWayAround() {
- assertFalse(new ResourceFilter(TOPIC, "*")
- .matches(new Resource(TOPIC, "Name")));
- }
-}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/auth/DefaultKafkaPrincipalBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/security/auth/DefaultKafkaPrincipalBuilderTest.java
index d458d11..73a03ab 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/auth/DefaultKafkaPrincipalBuilderTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/auth/DefaultKafkaPrincipalBuilderTest.java
@@ -18,8 +18,6 @@ package org.apache.kafka.common.security.auth;
import javax.security.auth.x500.X500Principal;
import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.network.Authenticator;
-import org.apache.kafka.common.network.TransportLayer;
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder;
import org.apache.kafka.common.security.kerberos.KerberosShortNamer;
import org.apache.kafka.common.security.scram.internals.ScramMechanism;
@@ -42,59 +40,10 @@ import static org.mockito.Mockito.when;
public class DefaultKafkaPrincipalBuilderTest {
@Test
- @SuppressWarnings("deprecation")
- public void testUseOldPrincipalBuilderForPlaintextIfProvided() throws Exception {
- TransportLayer transportLayer = mock(TransportLayer.class);
- Authenticator authenticator = mock(Authenticator.class);
- PrincipalBuilder oldPrincipalBuilder = mock(PrincipalBuilder.class);
-
- when(oldPrincipalBuilder.buildPrincipal(any(), any())).thenReturn(new DummyPrincipal("foo"));
-
- DefaultKafkaPrincipalBuilder builder = DefaultKafkaPrincipalBuilder.fromOldPrincipalBuilder(authenticator,
- transportLayer, oldPrincipalBuilder, null);
-
- KafkaPrincipal principal = builder.build(new PlaintextAuthenticationContext(
- InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name()));
- assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
- assertEquals("foo", principal.getName());
-
- builder.close();
-
- verify(oldPrincipalBuilder).buildPrincipal(transportLayer, authenticator);
- verify(oldPrincipalBuilder).close();
- }
-
- @Test
public void testReturnAnonymousPrincipalForPlaintext() throws Exception {
- try (DefaultKafkaPrincipalBuilder builder = new DefaultKafkaPrincipalBuilder(null, null)) {
- assertEquals(KafkaPrincipal.ANONYMOUS, builder.build(
- new PlaintextAuthenticationContext(InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name())));
- }
- }
-
- @Test
- @SuppressWarnings("deprecation")
- public void testUseOldPrincipalBuilderForSslIfProvided() throws Exception {
- TransportLayer transportLayer = mock(TransportLayer.class);
- Authenticator authenticator = mock(Authenticator.class);
- PrincipalBuilder oldPrincipalBuilder = mock(PrincipalBuilder.class);
- SSLSession session = mock(SSLSession.class);
-
- when(oldPrincipalBuilder.buildPrincipal(any(), any()))
- .thenReturn(new DummyPrincipal("foo"));
-
- DefaultKafkaPrincipalBuilder builder = DefaultKafkaPrincipalBuilder.fromOldPrincipalBuilder(authenticator,
- transportLayer, oldPrincipalBuilder, null);
-
- KafkaPrincipal principal = builder.build(
- new SslAuthenticationContext(session, InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name()));
- assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
- assertEquals("foo", principal.getName());
-
- builder.close();
-
- verify(oldPrincipalBuilder).buildPrincipal(transportLayer, authenticator);
- verify(oldPrincipalBuilder).close();
+ DefaultKafkaPrincipalBuilder builder = new DefaultKafkaPrincipalBuilder(null, null);
+ assertEquals(KafkaPrincipal.ANONYMOUS, builder.build(
+ new PlaintextAuthenticationContext(InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name())));
}
@Test
@@ -110,8 +59,6 @@ public class DefaultKafkaPrincipalBuilderTest {
assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
assertEquals("foo", principal.getName());
- builder.close();
-
verify(session, atLeastOnce()).getPeerPrincipal();
}
@@ -127,7 +74,6 @@ public class DefaultKafkaPrincipalBuilderTest {
new SslAuthenticationContext(session, InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name()));
assertEquals(KafkaPrincipal.ANONYMOUS, principal);
- builder.close();
verify(session, atLeastOnce()).getPeerPrincipal();
}
@@ -165,7 +111,6 @@ public class DefaultKafkaPrincipalBuilderTest {
principal = builder.build(sslContext);
assertEquals("OU=JavaSoft,O=Sun Microsystems,C=US", principal.getName());
- builder.close();
verify(session, times(4)).getPeerPrincipal();
}
@@ -183,8 +128,6 @@ public class DefaultKafkaPrincipalBuilderTest {
assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
assertEquals("foo", principal.getName());
- builder.close();
-
verify(server, atLeastOnce()).getMechanismName();
verify(server, atLeastOnce()).getAuthorizationID();
}
@@ -205,8 +148,6 @@ public class DefaultKafkaPrincipalBuilderTest {
assertEquals(KafkaPrincipal.USER_TYPE, principal.getPrincipalType());
assertEquals("foo", principal.getName());
- builder.close();
-
verify(server, atLeastOnce()).getMechanismName();
verify(server, atLeastOnce()).getAuthorizationID();
verify(kerberosShortNamer, atLeastOnce()).shortName(any());
@@ -232,8 +173,6 @@ public class DefaultKafkaPrincipalBuilderTest {
KafkaPrincipal deserializedPrincipal = builder.deserialize(serializedPrincipal);
assertEquals(principal, deserializedPrincipal);
- builder.close();
-
verify(server, atLeastOnce()).getMechanismName();
verify(server, atLeastOnce()).getAuthorizationID();
verify(kerberosShortNamer, atLeastOnce()).shortName(any());
diff --git a/docs/upgrade.html b/docs/upgrade.html
index 9485db2..2e11115 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -28,6 +28,9 @@
<li>The Streams API removed all deprecated APIs that were deprecated in version 2.5.0 or earlier.
For a complete list of removed APIs compare the detailed Kafka Streams upgrade notes.</li>
<li>The deprecated <code>Metric#value()</code> method was removed (<a href="https://issues.apache.org/jira/browse/KAFKA-12573">KAFKA-12573</a>).</li>
+ <li>Deprecated security classes were removed: <code>PrincipalBuilder</code>, <code>DefaultPrincipalBuilder</code> and <code>ResourceFilter</code>.
+ Furthermore, deprecated constants and constructors were removed from <code>SslConfigs</code>, <code>SaslConfigs</code>,
+ <code>AclBinding</code> and <code>AclBindingFilter</code>.</li>
</ul>
<h5><a id="upgrade_280_notable" href="#upgrade_280_notable">Notable changes in 2.8.0</a></h5>