-
Notifications
You must be signed in to change notification settings - Fork 241
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(auto_balancer): support client auth config (#1138)
* feat(auto_balancer): support client auth config Signed-off-by: Shichao Nie <[email protected]> * test(auto_balancer): fix unit tests Signed-off-by: Shichao Nie <[email protected]> --------- Signed-off-by: Shichao Nie <[email protected]>
- Loading branch information
Showing
7 changed files
with
168 additions
and
58 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
68 changes: 68 additions & 0 deletions
68
core/src/main/java/kafka/autobalancer/config/AutoBalancerConfigUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
/* | ||
* Copyright 2024, AutoMQ CO.,LTD. | ||
* | ||
* Use of this software is governed by the Business Source License | ||
* included in the file BSL.md | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package kafka.autobalancer.config; | ||
|
||
import org.apache.kafka.clients.CommonClientConfigs; | ||
import org.apache.kafka.clients.admin.AdminClientConfig; | ||
import org.apache.kafka.common.config.ConfigException; | ||
import org.apache.kafka.common.config.SaslConfigs; | ||
import org.apache.kafka.common.config.types.Password; | ||
|
||
import java.util.Properties; | ||
|
||
/** | ||
* This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsUtils. | ||
* Copyright 2020 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). | ||
*/ | ||
public class AutoBalancerConfigUtils { | ||
|
||
/** | ||
* Parse AdminClient configs based on the given {@link AutoBalancerConfig configs}. | ||
* | ||
* @param clientConfigs Configs that will be return with SSL configs. | ||
* @param configs Configs to be used for parsing AdminClient SSL configs. | ||
*/ | ||
public static void addSslConfigs(Properties clientConfigs, AutoBalancerConfig configs) { | ||
// Add security protocol (if specified). | ||
try { | ||
String securityProtocol = configs.getString(AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SECURITY_PROTOCOL); | ||
clientConfigs.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); | ||
setStringConfigIfExists(configs, clientConfigs, AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SECURITY_PROTOCOL, | ||
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); | ||
setStringConfigIfExists(configs, clientConfigs, AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SASL_MECHANISM, | ||
SaslConfigs.SASL_MECHANISM); | ||
setPasswordConfigIfExists(configs, clientConfigs, AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SASL_JAAS_CONFIG, | ||
SaslConfigs.SASL_JAAS_CONFIG); | ||
} catch (ConfigException ce) { | ||
// let it go. | ||
} | ||
} | ||
|
||
private static void setPasswordConfigIfExists(AutoBalancerConfig configs, Properties props, String name, String originalName) { | ||
try { | ||
Password pwd = configs.getPassword(name); | ||
if (pwd != null) { | ||
props.put(originalName, pwd); | ||
} | ||
} catch (ConfigException ce) { | ||
// let it go. | ||
} | ||
} | ||
|
||
private static void setStringConfigIfExists(AutoBalancerConfig configs, Properties props, String name, String originalName) { | ||
try { | ||
props.put(originalName, configs.getString(name)); | ||
} catch (ConfigException ce) { | ||
// let it go. | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
62 changes: 62 additions & 0 deletions
62
core/src/test/java/kafka/autobalancer/config/AutoBalancerConfigTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
/* | ||
* Copyright 2024, AutoMQ CO.,LTD. | ||
* | ||
* Use of this software is governed by the Business Source License | ||
* included in the file BSL.md | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package kafka.autobalancer.config; | ||
|
||
import org.apache.kafka.clients.admin.AdminClientConfig; | ||
import org.apache.kafka.common.config.ConfigException; | ||
import org.apache.kafka.common.config.SaslConfigs; | ||
import org.apache.kafka.common.config.types.Password; | ||
import org.apache.kafka.common.security.auth.SecurityProtocol; | ||
import org.junit.jupiter.api.Assertions; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Properties; | ||
|
||
public class AutoBalancerConfigTest { | ||
|
||
@Test | ||
public void testNoPredefinedConfig() { | ||
Map<String, String> props = new HashMap<>(); | ||
props.put(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG, "test-topic"); | ||
props.put(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG, "2"); | ||
props.put("some.other.config", "some-value"); | ||
AutoBalancerConfig config = new AutoBalancerConfig(props, false); | ||
Assertions.assertEquals("test-topic", config.getString(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG)); | ||
Assertions.assertEquals(2, config.getInt(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG)); | ||
Assertions.assertThrowsExactly(ConfigException.class, () -> config.getString("some.other.config")); | ||
} | ||
|
||
@Test | ||
public void testSSLConfig() { | ||
Map<String, Object> props = new HashMap<>(); | ||
props.put(AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name); | ||
props.put(AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SASL_MECHANISM, "PLAIN"); | ||
Password pwd = new Password("jaas-config"); | ||
props.put(AutoBalancerConfig.AUTO_BALANCER_CLIENT_AUTH_SASL_JAAS_CONFIG, pwd); | ||
|
||
AutoBalancerMetricsReporterConfig config = new AutoBalancerMetricsReporterConfig(props, false); | ||
Properties clientConfig = new Properties(); | ||
AutoBalancerConfigUtils.addSslConfigs(clientConfig, config); | ||
Assertions.assertEquals(SecurityProtocol.SASL_PLAINTEXT.name, clientConfig.getProperty(AdminClientConfig.SECURITY_PROTOCOL_CONFIG)); | ||
Assertions.assertEquals("PLAIN", clientConfig.getProperty(SaslConfigs.SASL_MECHANISM)); | ||
Assertions.assertEquals(pwd, clientConfig.get(SaslConfigs.SASL_JAAS_CONFIG)); | ||
|
||
AutoBalancerControllerConfig ctlConfig = new AutoBalancerControllerConfig(props, false); | ||
clientConfig = new Properties(); | ||
AutoBalancerConfigUtils.addSslConfigs(clientConfig, ctlConfig); | ||
Assertions.assertEquals(SecurityProtocol.SASL_PLAINTEXT.name, clientConfig.getProperty(AdminClientConfig.SECURITY_PROTOCOL_CONFIG)); | ||
Assertions.assertEquals("PLAIN", clientConfig.getProperty(SaslConfigs.SASL_MECHANISM)); | ||
Assertions.assertEquals(pwd, clientConfig.get(SaslConfigs.SASL_JAAS_CONFIG)); | ||
} | ||
} |