kirktrue commented on code in PR #16899: URL: https://github.com/apache/kafka/pull/16899#discussion_r1745840634
########## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ########## @@ -675,6 +688,7 @@ protected Map<String, Object> postProcessParsedConfig(final Map<String, Object> maybeOverrideClientId(refinedConfigs); maybeOverrideEnableAutoCommit(refinedConfigs); checkGroupRemoteAssignor(); + checkUnsupportConfigsWithConsumerProtocol(); Review Comment: We can call the more generalized `checkUnsupportedConfigs()` method directly: ```suggestion checkUnsupportedConfigs(GroupProtocol.CLASSIC, CLASSIC_PROTOCOL_UNSUPPORTED_CONFIGS); checkUnsupportedConfigs(GroupProtocol.CONSUMER, CONSUMER_PROTOCOL_UNSUPPORTED_CONFIGS); ``` ########## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ########## @@ -722,11 +736,39 @@ private void maybeOverrideEnableAutoCommit(Map<String, Object> configs) { } private void checkGroupRemoteAssignor() { - if (getString(GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(GroupProtocol.CLASSIC.name()) && getString(GROUP_REMOTE_ASSIGNOR_CONFIG) != null && !getString(GROUP_REMOTE_ASSIGNOR_CONFIG).isEmpty()) { + if (isClassicProtocol() && getString(GROUP_REMOTE_ASSIGNOR_CONFIG) != null && !getString(GROUP_REMOTE_ASSIGNOR_CONFIG).isEmpty()) { throw new ConfigException(GROUP_REMOTE_ASSIGNOR_CONFIG + " cannot be set when " + GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CLASSIC.name()); } } + private void checkUnsupportConfigsWithConsumerProtocol() { + if (isConsumerProtocol()) { + CONSUMER_PROTOCOL_UNSUPPORT_CONFIGS.forEach(configName -> { + Object config = originals().get(configName); + if (config != null && !Utils.isBlank(config.toString())) { + throw new ConfigException(configName + " cannot be set when " + GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CONSUMER.name()); + } + }); + } + } + + @Override + protected void clearDefaultValues() { + CONSUMER_PROTOCOL_UNSUPPORT_CONFIGS.forEach(configName -> { + if (originals().containsKey(configName)) { + super.clearConfig(configName); + } + }); + } + + private boolean isClassicProtocol() { + return getString(GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(GroupProtocol.CLASSIC.name()); + } + + private boolean isConsumerProtocol() { + return getString(GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(GroupProtocol.CONSUMER.name()); + } + Review Comment: ```suggestion private void checkUnsupportedConfigs(GroupProtocol groupProtocol, List<String> unsupportedConfigs) { if (getString(GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(groupProtocol.name())) { unsupportedConfigs.forEach(configName -> { Object config = originals().get(configName); if (config != null && !Utils.isBlank(config.toString())) { throw new ConfigException(configName + " cannot be set when " + GROUP_PROTOCOL_CONFIG + "=" + groupProtocol.name()); } else { super.clearConfig(configName); } }); } } ``` ########## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ########## @@ -384,7 +385,19 @@ public class ConsumerConfig extends AbstractConfig { private static final String SECURITY_PROVIDERS_DOC = SecurityConfig.SECURITY_PROVIDERS_DOC; private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); + private static final List<Class<? extends AbstractPartitionAssignor>> PARTITION_ASSIGNOR_DEFAULT_VALUE = + Collections.unmodifiableList(Arrays.asList(RangeAssignor.class, CooperativeStickyAssignor.class)); + /** + * A list of configuration keys for consumer protocol not supported. we should check the input string and clean up the + * default value. + */ + private static final List<String> CONSUMER_PROTOCOL_UNSUPPORT_CONFIGS = Collections.unmodifiableList(Arrays.asList( + PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + HEARTBEAT_INTERVAL_MS_CONFIG, + SESSION_TIMEOUT_MS_CONFIG + )); + Review Comment: ```suggestion /** * A list of configuration keys for CLASSIC protocol not supported. we should check the input string and clean up the * default value. */ private static final List<String> CLASSIC_PROTOCOL_UNSUPPORT_CONFIGS = Collections.unmodifiableList(Arrays.asList( GROUP_REMOTE_ASSIGNOR_CONFIG )); /** * A list of configuration keys for consumer protocol not supported. we should check the input string and clean up the * default value. */ private static final List<String> CONSUMER_PROTOCOL_UNSUPPORT_CONFIGS = Collections.unmodifiableList(Arrays.asList( PARTITION_ASSIGNMENT_STRATEGY_CONFIG, HEARTBEAT_INTERVAL_MS_CONFIG, SESSION_TIMEOUT_MS_CONFIG )); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org