kirktrue commented on code in PR #16899: URL: https://github.com/apache/kafka/pull/16899#discussion_r1823433917
########## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ########## @@ -379,7 +374,24 @@ 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 = + List.of(RangeAssignor.class, CooperativeStickyAssignor.class); + /** + * 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_UNSUPPORTED_CONFIGS = Collections.singletonList( + 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_UNSUPPORTED_CONFIGS = + List.of(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, HEARTBEAT_INTERVAL_MS_CONFIG, SESSION_TIMEOUT_MS_CONFIG); + Review Comment: Can we change this to a `Map<GroupProtocol, Set<String>> GROUP_PROTOCOL_UNSUPPORTED_CONFIGS` to make it a little more concise? ########## core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala: ########## @@ -83,8 +83,10 @@ abstract class BaseConsumerTest extends AbstractConsumerTest { @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testCoordinatorFailover(quorum: String, groupProtocol: String): Unit = { val listener = new TestConsumerReassignmentListener() - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "5001") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + if (groupProtocol.contains("CONSUMER")) { Review Comment: Why do we only set these for the new consumer? ########## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ########## @@ -61,12 +61,7 @@ public class ConsumerConfig extends AbstractConfig { // a list contains all the assignor names that only assign subscribed topics to consumer. Should be updated when new assignor added. // This is to help optimize ConsumerCoordinator#performAssignment method public static final List<String> ASSIGN_FROM_SUBSCRIBED_ASSIGNORS = - Collections.unmodifiableList(Arrays.asList( - RANGE_ASSIGNOR_NAME, - ROUNDROBIN_ASSIGNOR_NAME, - STICKY_ASSIGNOR_NAME, - COOPERATIVE_STICKY_ASSIGNOR_NAME - )); + List.of(RANGE_ASSIGNOR_NAME, ROUNDROBIN_ASSIGNOR_NAME, STICKY_ASSIGNOR_NAME, COOPERATIVE_STICKY_ASSIGNOR_NAME); Review Comment: Can we keep the configuration on each line like it is now? That allows the list to expand or contract without messing with line wrapping. But this is good: ```diff - Collections.unmodifiableList(Arrays.asList( + List.of( ``` -- 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