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

Reply via email to