-
Notifications
You must be signed in to change notification settings - Fork 14.2k
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
Changes from 5 commits
d3dca3c
1dd26e0
764abc8
ab68360
f766cbf
e58bfc7
e0d488a
5d7e54a
a6242de
4882842
ed254f0
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 |
---|---|---|
|
@@ -38,6 +38,7 @@ | |
import org.apache.kafka.common.MetricName; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.Uuid; | ||
import org.apache.kafka.common.errors.InvalidConfigurationException; | ||
import org.apache.kafka.common.metrics.KafkaMetric; | ||
import org.apache.kafka.common.metrics.MetricsReporter; | ||
import org.apache.kafka.common.serialization.StringDeserializer; | ||
|
@@ -58,10 +59,13 @@ | |
import java.util.UUID; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.stream.Collectors; | ||
import java.util.stream.Stream; | ||
|
||
import static java.util.Arrays.asList; | ||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.junit.jupiter.api.Assertions.assertNotNull; | ||
import static org.junit.jupiter.api.Assertions.assertNull; | ||
import static org.junit.jupiter.api.Assertions.assertThrows; | ||
import static org.junit.jupiter.api.Assertions.assertTrue; | ||
|
||
@ExtendWith(value = ClusterTestExtensions.class) | ||
|
@@ -120,6 +124,20 @@ public void testClientInstanceId(ClusterInstance clusterInstance) throws Interru | |
} | ||
} | ||
|
||
private static String[] toArray(List<String>... lists) { | ||
return Stream.of(lists).flatMap(List::stream).toArray(String[]::new); | ||
} | ||
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. nit: would be better to place the 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. Thanks. I will notice in future. |
||
@ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) | ||
public void testIntervalMsParser(ClusterInstance clusterInstance) { | ||
List<String> alterOpts = asList("--bootstrap-server", clusterInstance.bootstrapServers(), | ||
"--alter", "--entity-type", "client-metrics", "--entity-name", "test", "--add-config", "interval.ms=bbb"); | ||
try (Admin client = clusterInstance.createAdminClient()) { | ||
ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(alterOpts)); | ||
|
||
Throwable e = assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, addOpts)); | ||
assertTrue(e.getMessage().contains(InvalidConfigurationException.class.getSimpleName())); | ||
} | ||
} | ||
|
||
@ClusterTest(types = Type.KRAFT) | ||
public void testMetrics(ClusterInstance clusterInstance) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -123,6 +123,7 @@ public static void validate(String subscriptionName, Properties properties) { | |
validateProperties(properties); | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
private static void validateProperties(Properties properties) { | ||
// Make sure that all the properties are valid | ||
properties.forEach((key, value) -> { | ||
|
@@ -133,8 +134,8 @@ private static void validateProperties(Properties properties) { | |
|
||
// 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)); | ||
if (pushIntervalMs < MIN_INTERVAL_MS || pushIntervalMs > MAX_INTERVAL_MS) { | ||
Integer pushIntervalMs = (Integer) ConfigDef.parseType(PUSH_INTERVAL_MS, properties.getProperty(PUSH_INTERVAL_MS), Type.INT); | ||
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. If I get it right, the issue we are trying to solve is regarding the correct exception when config is not as INT or expected? If that's the case then either of the 3 approach I think would be better:
I would have chosen option 3. Also if you detail on the prolem satetment in PR description or jira then I can help better. But as far as I think, option 3 might solve your issue. 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. I saw the updated jira, thank you. Let me know if option 3 works for you, ideally should. 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. Hi @apoorvmittal10 , sorry for late response. |
||
if (pushIntervalMs == null || pushIntervalMs < MIN_INTERVAL_MS || pushIntervalMs > MAX_INTERVAL_MS) { | ||
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. Hmmm, what happens when we use 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. Hi @apoorvmittal10, thanks for the review and sorry for late response. If kafka-client-metrics.sh doesn't have pushIntervalMs, it should not enter this block (check in 136). |
||
String msg = String.format("Invalid value %s for %s, interval must be between 100 and 3600000 (1 hour)", | ||
pushIntervalMs, PUSH_INTERVAL_MS); | ||
throw new InvalidRequestException(msg); | ||
|
@@ -143,7 +144,7 @@ private static void validateProperties(Properties properties) { | |
|
||
// Make sure that client match patterns are valid by parsing them. | ||
if (properties.containsKey(CLIENT_MATCH_PATTERN)) { | ||
TaiJuWu marked this conversation as resolved.
Show resolved
Hide resolved
|
||
List<String> patterns = Arrays.asList(properties.getProperty(CLIENT_MATCH_PATTERN).split(",")); | ||
List<String> patterns = (List<String>) ConfigDef.parseType(CLIENT_MATCH_PATTERN, properties.getProperty(CLIENT_MATCH_PATTERN), Type.LIST); | ||
// Parse the client matching patterns to validate if the patterns are valid. | ||
parseMatchingPatterns(patterns); | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: a line break