Skip to content
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

KAFKA-17525: Convert the UnknownServerException to InvalidRequestException when altering client-metrics config at runtime #17168

Merged
merged 11 commits into from
Oct 4, 2024
12 changes: 12 additions & 0 deletions core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.kafka.server.config.ZooKeeperInternals;
import org.apache.kafka.test.TestUtils;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.platform.commons.util.StringUtils;

Expand Down Expand Up @@ -528,6 +529,17 @@ public void testUpdatePerBrokerConfigInKRaftThenShouldFail() {
}
}

@ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT})
public void testIntervalMsParser() {
alterOpts = asList("--bootstrap-server", cluster.bootstrapServers(),
"--alter", "--entity-type", "client-metrics", "--entity-name", "test");
try (Admin client = cluster.createAdminClient()) {
Throwable e = assertThrows(ExecutionException.class, () ->
alterConfigWithKraft(client, Optional.of(defaultBrokerId), singletonMap("interval.ms", "aaa")));
Assertions.assertTrue(e.getMessage().contains("InvalidConfigurationException"));
}
}

private void assertNonZeroStatusExit(Stream<String> args, Consumer<String> checkErrOut) {
AtomicReference<Integer> exitStatus = new AtomicReference<>();
Exit.setExitProcedure((status, __) -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,9 +131,11 @@ private static void validateProperties(Properties properties) {
}
});

ClientMetricsConfigs configs = new ClientMetricsConfigs(properties);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What happens if there are configs having no default value?

Copy link
Contributor Author

@TaiJuWu TaiJuWu Sep 14, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @chia7712 , thanks for review.

The situation won't happen because Line 120 checks the properties which are defined at Line 102 and they have default value except PUSH_INTERVAL_MS.

Copy link
Collaborator

@apoorvmittal10 apoorvmittal10 Sep 25, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not convinced instantiating ClientMetricsConfigs(properties) in the validate method. The idea with validate is to create ClientMetricsConfigs instance only after post validating the configs are correct.

I understand we want to thrown invalid request exception when configs are not correct and as per the changes I am assuming that the unknown server occurs when the int config is specified as non-int, correct? If that's the only scenario then we can handle directly. Am I reading something wrong here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not convinced instantiating ClientMetricsConfigs(properties) in the validate method. The idea with validate is to create ClientMetricsConfigs instance only after post validating the configs are correct.

I understand we want to thrown invalid request exception when configs are not correct and as per the changes I am assuming that the unknown server occurs when the int config is specified as non-int, correct? If that's the only scenario then we can handle directly. Am I reading something wrong here?

Yes, you are right and I updated this part by utilize ConfigDef.parseType.
PTAL.


// Make sure that push interval is between 100ms and 1 hour.
if (properties.containsKey(PUSH_INTERVAL_MS)) {
int pushIntervalMs = Integer.parseInt(properties.getProperty(PUSH_INTERVAL_MS));
int pushIntervalMs = configs.getInt(PUSH_INTERVAL_MS);
if (pushIntervalMs < MIN_INTERVAL_MS || pushIntervalMs > MAX_INTERVAL_MS) {
String msg = String.format("Invalid value %s for %s, interval must be between 100 and 3600000 (1 hour)",
pushIntervalMs, PUSH_INTERVAL_MS);
Expand All @@ -143,7 +145,7 @@ private static void validateProperties(Properties properties) {

// Make sure that client match patterns are valid by parsing them.
if (properties.containsKey(CLIENT_MATCH_PATTERN)) {
List<String> patterns = Arrays.asList(properties.getProperty(CLIENT_MATCH_PATTERN).split(","));
List<String> patterns = configs.getList(CLIENT_MATCH_PATTERN);
// Parse the client matching patterns to validate if the patterns are valid.
parseMatchingPatterns(patterns);
}
Expand Down