kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385631654
########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -222,42 +214,74 @@ public void cleanup() { } } - @Test - public void testMetricsReporterAutoGeneratedClientId() { + private static Collection<Arguments> bothGroupProtocols() { + return Arrays.stream(GroupProtocol.values()).map(Arguments::of).collect(Collectors.toList()); + } + + /** + * A given test may choose to use the {@link GroupProtocol#GENERIC generic group protocol} for a number of reasons. + * Among the reasons for a test to do so is because it... + * + * <ul> + * <li> + * ...exercises rebalancing logic that is not yet implemented in the + * {@link GroupProtocol#CONSUMER consumer group protocol}. + * </li> + * <li>...includes topic metadata that is not yet implemented in the consumer group protocol.</li> + * <li>...fails, possibly due to the omission of functionality in the consumer group protocol.</li> + * <li>...uses logic, timing, etc. that are not applicable to the consumer group protocol.</li> + * </ul> + * + * Less than half of the tests for the consumer group protocol pass as of now, but it's very tedious to + * investigate at this point due to known bugs and missing functionality. + */ + private static Collection<Arguments> genericGroupProtocolOnly() { + return Collections.singleton(Arguments.of(GroupProtocol.GENERIC)); + } + + @ParameterizedTest + @MethodSource("bothGroupProtocols") Review Comment: Done. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java: ########## @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import org.apache.kafka.clients.consumer.GroupProtocol; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +import java.util.List; +import java.util.Locale; + +/** + * {@code ConsumerDelegateCreator} implements a quasi-factory pattern to allow the caller to remain unaware of the + * underlying {@link Consumer} implementation that is created. This provides the means by which {@link KafkaConsumer} + * can remain the top-level facade for implementations, but allow different implementations to co-exist under + * the covers. + * + * <p/> + * + * The current logic for the {@code ConsumerCreator} inspects the incoming configuration and determines if + * it is using the new KIP-848 consumer protocol or if it should fall back to the existing, legacy group protocol. Review Comment: Done. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegate.java: ########## @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.Timer; + +/** + * This extension interface provides a handful of methods to expose internals of the {@link Consumer} for + * various tests. + */ +public interface ConsumerDelegate<K, V> extends Consumer<K, V> { + + String getClientId(); Review Comment: Done. -- 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