- 
                Notifications
    You must be signed in to change notification settings 
- Fork 14.8k
KAFKA-19824: New AllowlistConnectorClientConfigOverridePolicy (KIP-1188) #20750
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,64 @@ | ||
| /* | ||
| * 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.connect.connector.policy; | ||
|  | ||
| import org.apache.kafka.common.config.AbstractConfig; | ||
| import org.apache.kafka.common.config.ConfigDef; | ||
| import org.apache.kafka.common.config.ConfigValue; | ||
|  | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|  | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|  | ||
| /** | ||
| * Allows only client configurations specified via <code>connector.client.config.override.allowlist</code> to be | ||
| * overridden by connectors. By default, <code>connector.client.config.override.allowlist</code> is empty so connectors | ||
| * can't override any client configurations. | ||
| */ | ||
| public class AllowlistConnectorClientConfigOverridePolicy extends AbstractConnectorClientConfigOverridePolicy { | ||
|  | ||
| public static final String ALLOWLIST_CONFIG = "connector.client.config.override.allowlist"; | ||
|  | ||
| private static final Logger LOGGER = LoggerFactory.getLogger(AllowlistConnectorClientConfigOverridePolicy.class); | ||
| private static final List<String> ALLOWLIST_CONFIG_DEFAULT = List.of(); | ||
| private static final String ALLOWLIST_CONFIG_DOC = "List of client configurations that can be overridden by " + | ||
| "connectors. If empty, connectors can't override any client configurations."; | ||
| private static final ConfigDef CONFIG_DEF = new ConfigDef() | ||
| .define(ALLOWLIST_CONFIG, ConfigDef.Type.LIST, ALLOWLIST_CONFIG_DEFAULT, ConfigDef.Importance.MEDIUM, ALLOWLIST_CONFIG_DOC); | ||
|  | ||
| private List<String> allowlist = ALLOWLIST_CONFIG_DEFAULT; | ||
|  | ||
| @Override | ||
| protected String policyName() { | ||
| return "Allowlist"; | ||
| } | ||
|  | ||
| @Override | ||
| protected boolean isAllowed(ConfigValue configValue) { | ||
| return allowlist.contains(configValue.name()); | ||
| } | ||
|  | ||
| @Override | ||
| public void configure(Map<String, ?> configs) { | ||
| AbstractConfig config = new AbstractConfig(CONFIG_DEF, configs); | ||
| allowlist = config.getList(ALLOWLIST_CONFIG); | ||
| LOGGER.info("Setting up Allowlist policy for ConnectorClientConfigOverride. This will allow the following client configurations" | ||
| + " to be overridden. {}", allowlist); | ||
| } | ||
| } | 
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
|  | @@ -32,7 +32,9 @@ | |
| /** | ||
| * Allows all {@code sasl} configurations to be overridden via the connector configs by setting {@code connector.client.config.override.policy} to | ||
| * {@code Principal}. This allows to set a principal per connector. | ||
| * @deprecated Use {@link AllowlistConnectorClientConfigOverridePolicy} instead. | ||
| */ | ||
| @Deprecated(since = " 4.2", forRemoval = true) | ||
| public class PrincipalConnectorClientConfigOverridePolicy extends AbstractConnectorClientConfigOverridePolicy { | ||
| There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @mimaison Do you have time to open a minor PR to fix the warning? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good spot! I opened #20793 | ||
| private static final Logger log = LoggerFactory.getLogger(PrincipalConnectorClientConfigOverridePolicy.class); | ||
|  | ||
|  | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
|  | @@ -27,6 +27,7 @@ | |
| import org.apache.kafka.common.metrics.JmxReporter; | ||
| import org.apache.kafka.common.metrics.Sensor; | ||
| import org.apache.kafka.common.utils.Utils; | ||
| import org.apache.kafka.connect.connector.policy.AllowlistConnectorClientConfigOverridePolicy; | ||
| import org.apache.kafka.connect.errors.ConnectException; | ||
| import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; | ||
| import org.apache.kafka.connect.runtime.rest.RestServerConfig; | ||
|  | @@ -159,8 +160,9 @@ public class WorkerConfig extends AbstractConfig { | |
| public static final String CONNECTOR_CLIENT_POLICY_CLASS_DOC = | ||
| "Class name or alias of implementation of <code>ConnectorClientConfigOverridePolicy</code>. Defines what client configurations can be " | ||
| + "overridden by the connector. The default implementation is <code>All</code>, meaning connector configurations can override all client properties. " | ||
|          | ||
| + "The other possible policies in the framework include <code>None</code> to disallow connectors from overriding client properties, " | ||
| + "and <code>Principal</code> to allow connectors to override only client principals."; | ||
| + "The other possible policies in the framework include <code>Allowlist</code> to specify allowed configurations via " | ||
| + "<code>" + AllowlistConnectorClientConfigOverridePolicy.ALLOWLIST_CONFIG + "</code>, <code>None</code> to disallow connectors from overriding " | ||
| + "client properties, and <code>Principal</code> (now deprecated) to allow connectors to override only client principals."; | ||
| public static final String CONNECTOR_CLIENT_POLICY_CLASS_DEFAULT = "All"; | ||
|  | ||
|  | ||
|  | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,76 @@ | ||
| /* | ||
| * 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.connect.connector.policy; | ||
|  | ||
| import org.apache.kafka.clients.admin.AdminClientConfig; | ||
| import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
| import org.apache.kafka.clients.producer.ProducerConfig; | ||
|  | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
|  | ||
| import java.util.Collection; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.stream.Stream; | ||
|  | ||
| public class AllowlistConnectorClientConfigOverridePolicyTest extends BaseConnectorClientConfigOverridePolicyTest { | ||
|  | ||
| private static final List<String> ALL_CONFIGS = Stream.of( | ||
| ProducerConfig.configNames(), | ||
| ConsumerConfig.configNames(), | ||
| AdminClientConfig.configNames()) | ||
| .flatMap(Collection::stream) | ||
| .toList(); | ||
|  | ||
| private AllowlistConnectorClientConfigOverridePolicy policy; | ||
|  | ||
| @BeforeEach | ||
| public void setUp() { | ||
| policy = new AllowlistConnectorClientConfigOverridePolicy(); | ||
| } | ||
|  | ||
| @Override | ||
| protected ConnectorClientConfigOverridePolicy policyToTest() { | ||
| return policy; | ||
| } | ||
|  | ||
| @Test | ||
| public void testDenyAllByDefault() { | ||
| for (String config : ALL_CONFIGS) { | ||
| testInvalidOverride(Map.of(config, new Object())); | ||
| } | ||
| } | ||
|  | ||
| @Test | ||
| public void testAllowConfigs() { | ||
| Set<String> allowedConfigs = Set.of( | ||
| ProducerConfig.ACKS_CONFIG, | ||
| ConsumerConfig.CLIENT_ID_CONFIG, | ||
| AdminClientConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG | ||
| ); | ||
| policy.configure(Map.of(AllowlistConnectorClientConfigOverridePolicy.ALLOWLIST_CONFIG, String.join(",", allowedConfigs))); | ||
| for (String config : ALL_CONFIGS) { | ||
| if (!allowedConfigs.contains(config)) { | ||
| testInvalidOverride(Map.of(config, new Object())); | ||
| } else { | ||
| testValidOverride(Map.of(config, new Object())); | ||
| } | ||
| } | ||
| } | ||
| } | 
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
|  | @@ -32,6 +32,7 @@ | |
| import org.apache.kafka.common.utils.Time; | ||
| import org.apache.kafka.connect.connector.Connector; | ||
| import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; | ||
| import org.apache.kafka.connect.connector.policy.AllowlistConnectorClientConfigOverridePolicy; | ||
| import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; | ||
| import org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy; | ||
| import org.apache.kafka.connect.connector.policy.PrincipalConnectorClientConfigOverridePolicy; | ||
|  | @@ -696,6 +697,7 @@ private PluginDesc<Transformation<?>> transformationPluginDesc() { | |
| return new PluginDesc(SampleTransformation.class, "1.0", PluginType.TRANSFORMATION, classLoader); | ||
| } | ||
|  | ||
| @SuppressWarnings("removal") | ||
| @Test | ||
| public void testConfigValidationPrincipalOnlyOverride() { | ||
| final Class<? extends Connector> connectorClass = SampleSourceConnector.class; | ||
|  | @@ -788,6 +790,50 @@ public void testConfigValidationAllOverride() { | |
| verifyValidationIsolation(); | ||
| } | ||
|  | ||
| @Test | ||
| public void testConfigValidationAllowlistOverride() { | ||
| final Class<? extends Connector> connectorClass = SampleSourceConnector.class; | ||
| AllowlistConnectorClientConfigOverridePolicy policy = new AllowlistConnectorClientConfigOverridePolicy(); | ||
| policy.configure(Map.of(AllowlistConnectorClientConfigOverridePolicy.ALLOWLIST_CONFIG, "acks")); | ||
| AbstractHerder herder = createConfigValidationHerder(connectorClass, policy); | ||
|  | ||
| Map<String, String> config = new HashMap<>(); | ||
| config.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName()); | ||
| config.put(ConnectorConfig.NAME_CONFIG, "connector-name"); | ||
| config.put("required", "value"); // connector required config | ||
| String ackConfigKey = producerOverrideKey(ProducerConfig.ACKS_CONFIG); | ||
| String saslConfigKey = producerOverrideKey(SaslConfigs.SASL_JAAS_CONFIG); | ||
| config.put(ackConfigKey, "none"); | ||
| config.put(saslConfigKey, "jaas_config"); | ||
|  | ||
| ConfigInfos result = herder.validateConnectorConfig(config, s -> null, false); | ||
| assertEquals(ConnectorType.SOURCE, herder.connectorType(config)); | ||
|  | ||
| // We expect there to be errors due to sasl.jaas.config not being allowed Note that these assertions depend heavily on | ||
|          | ||
| // the config fields for SourceConnectorConfig, but we expect these to change rarely. | ||
| assertEquals(SampleSourceConnector.class.getName(), result.name()); | ||
| // Each transform also gets its own group | ||
| List<String> expectedGroups = List.of( | ||
| ConnectorConfig.COMMON_GROUP, | ||
| ConnectorConfig.TRANSFORMS_GROUP, | ||
| ConnectorConfig.PREDICATES_GROUP, | ||
| ConnectorConfig.ERROR_GROUP, | ||
| SourceConnectorConfig.TOPIC_CREATION_GROUP, | ||
| SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_GROUP, | ||
| SourceConnectorConfig.OFFSETS_TOPIC_GROUP | ||
| ); | ||
| assertEquals(expectedGroups, result.groups()); | ||
| assertEquals(1, result.errorCount()); | ||
| // Base connector config has 19 fields, connector's configs add 7, and 2 producer overrides | ||
| assertEquals(28, result.configs().size()); | ||
| assertTrue(result.configs().stream().anyMatch( | ||
| configInfo -> ackConfigKey.equals(configInfo.configValue().name()) && configInfo.configValue().errors().isEmpty())); | ||
| assertTrue(result.configs().stream().anyMatch( | ||
| configInfo -> saslConfigKey.equals(configInfo.configValue().name()) && !configInfo.configValue().errors().isEmpty())); | ||
|  | ||
| verifyValidationIsolation(); | ||
| } | ||
|  | ||
| static final class TestClientConfigOverridePolicy extends AllConnectorClientConfigOverridePolicy implements Monitorable { | ||
|  | ||
| private static MetricName metricName = null; | ||
|  | ||
Uh oh!
There was an error while loading. Please reload this page.