dajac commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1384566593
########## 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. Review Comment: nit: Should we make it clear that this interface is only used internally? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ########## @@ -247,12 +247,10 @@ private void closeInternal(final Duration timeout) { closeTimeout = timeout; wakeup(); - if (timeoutMs > 0) { - try { - join(timeoutMs); - } catch (InterruptedException e) { - log.error("Interrupted while waiting for consumer network thread to complete", e); - } + try { Review Comment: Why do we need this change? ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -835,16 +919,17 @@ public void testMissingOffsetNoResetPolicy() { assertThrows(NoOffsetForPartitionException.class, () -> consumer.poll(Duration.ZERO)); } - @Test - public void testResetToCommittedOffset() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: Same. Actually, it seems to me that all the tests relying on `consumer.assign` should work, no? ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -721,16 +802,17 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { assertEquals(55L, consumer.position(tp0)); } - @Test - public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: Why is this one only applied to the generic protocol? It seems that it should also work for the new one. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java: ########## @@ -0,0 +1,2554 @@ +/* + * 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.ApiVersions; +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.consumer.CommitFailedException; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerInterceptor; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.IsolationLevel; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.internals.ClusterResourceListeners; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.AppInfoParser; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; +import org.slf4j.event.Level; + +import java.net.InetSocketAddress; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Pattern; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createConsumerNetworkClient; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors; +import static org.apache.kafka.common.utils.Utils.closeQuietly; +import static org.apache.kafka.common.utils.Utils.isBlank; +import static org.apache.kafka.common.utils.Utils.join; +import static org.apache.kafka.common.utils.Utils.swallow; + +/** + * A client that consumes records from a Kafka cluster. + * <p> + * This client transparently handles the failure of Kafka brokers, and transparently adapts as topic partitions + * it fetches migrate within the cluster. This client also interacts with the broker to allow groups of + * consumers to load balance consumption using <a href="#consumergroups">consumer groups</a>. + * <p> + * The consumer maintains TCP connections to the necessary brokers to fetch data. + * Failure to close the consumer after use will leak these connections. + * The consumer is not thread-safe. See <a href="#multithreaded">Multi-threaded Processing</a> for more details. + * + * <h3>Cross-Version Compatibility</h3> + * This client can communicate with brokers that are version 0.10.0 or newer. Older or newer brokers may not support + * certain features. For example, 0.10.0 brokers do not support offsetsForTimes, because this feature was added + * in version 0.10.1. You will receive an {@link org.apache.kafka.common.errors.UnsupportedVersionException} + * when invoking an API that is not available on the running broker version. + * <p> + * + * <h3>Offsets and Consumer Position</h3> Review Comment: I wonder if we should just remove all the main/common javadoc from this class and keep it in KafkaConsumer. The issue that I see is that this one will get out of sync. What do you think? ########## 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. + */ Review Comment: This is kind of obvious. Should we remove it? In my opinion, we should aim for making all of them pass -- not in this PR, obviously -- and if one cannot, we should add a comment explaining why. ########## 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: nit: I understand that you reused the existing name here. However, we usually don't prefix getters with `get`. Should we fix it while here? ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -814,16 +897,17 @@ private void initMetadata(MockClient mockClient, Map<String, Integer> partitionC mockClient.updateMetadata(initialMetadata); } - @Test - public void testMissingOffsetNoResetPolicy() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: ditto. ########## 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: nit: `the new KIP-848 consumer protocol` -> `the new consumer group protocol (KIP-848)`? ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -757,8 +839,9 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit() { assertEquals(55, consumer.committed(Collections.singleton(tp0), Duration.ZERO).get(tp0).offset()); } - @Test - public void testFetchProgressWithMissingPartitionPosition() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: ditto. ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -1893,8 +2014,9 @@ private void consumerCloseTest(final long closeTimeoutMs, } } - @Test - public void testPartitionsForNonExistingTopic() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") + public void testPartitionsForNonExistingTopic(GroupProtocol groupProtocol) { Review Comment: I suppose that we will update all those tests to support the new consumer at some point, right? ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -857,16 +942,17 @@ public void testResetToCommittedOffset() { assertEquals(539L, consumer.position(tp0)); } - @Test - public void testResetUsingAutoResetPolicy() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: ditto. ########## 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(); + + Metrics metricsInternal(); Review Comment: This one was not there before. What's the reason for introducing it? I wonder if we should just keeping using `Consumer.metrics()`. ########## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ########## @@ -881,24 +967,26 @@ public void testResetUsingAutoResetPolicy() { assertEquals(50L, consumer.position(tp0)); } - @Test - public void testOffsetIsValidAfterSeek() { + @ParameterizedTest + @MethodSource("bothGroupProtocols") + public void testOffsetIsValidAfterSeek(GroupProtocol groupProtocol) { SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST); ConsumerMetadata metadata = createMetadata(subscription); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); - consumer = newConsumer(time, client, subscription, metadata, assignor, + consumer = newConsumer(groupProtocol, time, client, subscription, metadata, assignor, true, groupId, Optional.empty(), false); consumer.assign(singletonList(tp0)); consumer.seek(tp0, 20L); consumer.poll(Duration.ZERO); assertEquals(subscription.validPosition(tp0).offset, 20L); } - @Test - public void testCommitsFetchedDuringAssign() { + @ParameterizedTest + @MethodSource("genericGroupProtocolOnly") Review Comment: ditto. ########## 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: nit: The `EnumSource` is pretty handy for this: `@EnumSource(value = GroupProtocol.class, names = {"GENERIC", "CONSUMER"})`. Any reason not to use it? -- 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