From a5fa2ecaa16004b941fbe215f24f48b9b6d3a257 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Fri, 21 Mar 2025 15:00:17 +0000 Subject: [PATCH] KAFKA-19026: AlterConfigPolicy incompatibility between ZK mode and KRaft mode when using AlterConfigOp.OpType.SUBTRACT Modified ZkAdminManager.scala so that on OpType.SUBTRACT the policy receives the modified configs, as happens in KRaft mode. This similarly fixes the OpType.APPEND differences. Note that the policy behavior on OpType.DELETE is different when altering Broker and Topic resources. For topics the policy does not see a map entry, for brokers the config value is null. This was the existing behavior for KRaft and this commit does not change that. ClusterTest added. --- .../scala/kafka/server/ZkAdminManager.scala | 6 +- .../server/AlterConfigPolicyConfigsTest.java | 207 ++++++++++++++++++ 2 files changed, 211 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/kafka/server/AlterConfigPolicyConfigsTest.java diff --git a/core/src/main/scala/kafka/server/ZkAdminManager.scala b/core/src/main/scala/kafka/server/ZkAdminManager.scala index dc4c5d0652200..cb120ee8a1490 100644 --- a/core/src/main/scala/kafka/server/ZkAdminManager.scala +++ b/core/src/main/scala/kafka/server/ZkAdminManager.scala @@ -503,8 +503,6 @@ class ZkAdminManager(val config: KafkaConfig, def incrementalAlterConfigs(configs: Map[ConfigResource, Seq[AlterConfigOp]], validateOnly: Boolean): Map[ConfigResource, ApiError] = { configs.map { case (resource, alterConfigOps) => try { - val configEntriesMap = alterConfigOps.map(entry => (entry.configEntry.name, entry.configEntry.value)).toMap - resource.`type` match { case ConfigResource.Type.TOPIC => if (resource.name.isEmpty) { @@ -512,6 +510,8 @@ class ZkAdminManager(val config: KafkaConfig, } val configProps = adminZkClient.fetchEntityConfig(ConfigType.TOPIC, resource.name) prepareIncrementalConfigs(alterConfigOps, configProps, LogConfig.configKeys.asScala) + val configEntriesMap = alterConfigOps.map(entry => + (entry.configEntry.name, configProps.getProperty(entry.configEntry.name))).filter(x => x._2 != null).toMap alterTopicConfigs(resource, validateOnly, configProps, configEntriesMap) case ConfigResource.Type.BROKER => @@ -523,6 +523,8 @@ class ZkAdminManager(val config: KafkaConfig, val configProps = this.config.dynamicConfig.fromPersistentProps(persistentProps, perBrokerConfig) prepareIncrementalConfigs(alterConfigOps, configProps, KafkaConfig.configKeys) + val configEntriesMap = alterConfigOps.map(entry => + (entry.configEntry.name, configProps.getProperty(entry.configEntry.name))).toMap alterBrokerConfigs(resource, validateOnly, configProps, configEntriesMap) case resourceType => diff --git a/core/src/test/java/kafka/server/AlterConfigPolicyConfigsTest.java b/core/src/test/java/kafka/server/AlterConfigPolicyConfigsTest.java new file mode 100644 index 0000000000000..dd5ed32f69676 --- /dev/null +++ b/core/src/test/java/kafka/server/AlterConfigPolicyConfigsTest.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server; + +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterConfigProperty; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.junit.ClusterTestExtensions; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.PolicyViolationException; +import org.apache.kafka.server.policy.AlterConfigPolicy; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +import static org.apache.kafka.server.config.ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG; +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; + +@ClusterTestDefaults(serverProperties = { + @ClusterConfigProperty(key = ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, value = "kafka.server.AlterConfigPolicyConfigsTest$Policy"), +}) +@ExtendWith(value = ClusterTestExtensions.class) +public class AlterConfigPolicyConfigsTest { + + @BeforeEach + public void setUp() { + Policy.lastConfig = null; + } + + @ClusterTest + public void testPolicyAlterBrokerConfigSubtract(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + clusterInstance.waitForReadyBrokers(); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(SslConfigs.SSL_CIPHER_SUITES_CONFIG, "foo"), + AlterConfigOp.OpType.SUBTRACT); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.BROKER, "0"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("", Policy.lastConfig.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterBrokerConfigAppend(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + clusterInstance.waitForReadyBrokers(); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(SslConfigs.SSL_CIPHER_SUITES_CONFIG, "foo,bar"), + AlterConfigOp.OpType.APPEND); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.BROKER, "0"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("foo,bar", Policy.lastConfig.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterBrokerConfigSet(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + clusterInstance.waitForReadyBrokers(); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(SslConfigs.SSL_CIPHER_SUITES_CONFIG, "foo"), + AlterConfigOp.OpType.SET); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.BROKER, "0"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("foo", Policy.lastConfig.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterBrokerConfigDelete(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + clusterInstance.waitForReadyBrokers(); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(SslConfigs.SSL_CIPHER_SUITES_CONFIG, "unused"), + AlterConfigOp.OpType.DELETE); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.BROKER, "0"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertTrue(Policy.lastConfig.containsKey(SslConfigs.SSL_CIPHER_SUITES_CONFIG)); + assertNull(Policy.lastConfig.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterTopicConfigSubtract(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + admin.createTopics(Collections.singleton(new NewTopic("topic1", 1, (short) 1))).all().get(); + clusterInstance.waitForTopic("topic1", 1); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "foo"), + AlterConfigOp.OpType.SUBTRACT); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.TOPIC, "topic1"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("delete", Policy.lastConfig.get(TopicConfig.CLEANUP_POLICY_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterTopicConfigAppend(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + admin.createTopics(Collections.singleton(new NewTopic("topic1", 1, (short) 1))).all().get(); + clusterInstance.waitForTopic("topic1", 1); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "compact"), + AlterConfigOp.OpType.APPEND); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.TOPIC, "topic1"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("delete,compact", Policy.lastConfig.get(TopicConfig.CLEANUP_POLICY_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterTopicConfigSet(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + admin.createTopics(Collections.singleton(new NewTopic("topic1", 1, (short) 1))).all().get(); + clusterInstance.waitForTopic("topic1", 1); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "compact"), + AlterConfigOp.OpType.SET); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.TOPIC, "topic1"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertEquals("compact", Policy.lastConfig.get(TopicConfig.CLEANUP_POLICY_CONFIG)); + } + } + + @ClusterTest + public void testPolicyAlterTopicConfigDelete(ClusterInstance clusterInstance) throws Exception { + try (Admin admin = clusterInstance.createAdminClient()) { + admin.createTopics(Collections.singleton(new NewTopic("topic1", 1, (short) 1))).all().get(); + clusterInstance.waitForTopic("topic1", 1); + + AlterConfigOp alterConfigOp = new AlterConfigOp( + new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "unused"), + AlterConfigOp.OpType.DELETE); + Map> alterConfigs = Collections.singletonMap( + new ConfigResource(ConfigResource.Type.TOPIC, "topic1"), + Collections.singletonList(alterConfigOp)); + admin.incrementalAlterConfigs(alterConfigs).all().get(); + assertFalse(Policy.lastConfig.containsKey(TopicConfig.CLEANUP_POLICY_CONFIG)); + } + } + + public static class Policy implements AlterConfigPolicy { + public static Map lastConfig; + + @Override + public void validate(AlterConfigPolicy.RequestMetadata requestMetadata) throws PolicyViolationException { + assertNull(lastConfig); + lastConfig = requestMetadata.configs(); + } + + @Override + public void close() throws Exception {} + @Override + public void configure(Map configs) {} + } + +}