You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/06/30 09:09:32 UTC

[GitHub] [kafka] cadonna commented on a diff in pull request #12337: KAFKA-10199: Remove main consumer from store changelog reader

cadonna commented on code in PR #12337:
URL: https://github.com/apache/kafka/pull/12337#discussion_r910766856


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java:
##########
@@ -110,6 +112,44 @@ public static Map<MetricName, Metric> producerMetrics(final Collection<StreamsPr
         return result;
     }
 
+    /**
+     * @throws StreamsException if the consumer throws an exception
+     * @throws org.apache.kafka.common.errors.TimeoutException if the request times out
+     */
+    public static Map<TopicPartition, Long> fetchCommittedOffsets(final Set<TopicPartition> partitions,
+                                                                  final String groupId,
+                                                                  final Admin adminClient) {
+        if (partitions.isEmpty()) {
+            return Collections.emptyMap();
+        }
+
+        final Map<TopicPartition, Long> committedOffsets;
+        try {
+            // those which do not have a committed offset would default to 0
+            final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions();
+            options.topicPartitions(new ArrayList<>(partitions));
+            options.requireStable(true);
+            committedOffsets = adminClient.listConsumerGroupOffsets(groupId, options)
+                    .partitionsToOffsetAndMetadata().get().entrySet()
+                    .stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset()));
+        } catch (final InterruptedException e) {
+            LOG.warn("The committed offsets request failed due to interruption", e);
+            throw new StreamsException(String.format("Failed to retrieve end offsets for %s", partitions), e);
+        } catch (final ExecutionException e) {
+            if (e.getCause() instanceof TimeoutException) {
+                final TimeoutException exception = (TimeoutException) e.getCause();
+                LOG.warn("The committed offsets request timed out, try increasing the consumer client's default.api.timeout.ms", exception);
+                throw exception;
+            } else {
+                LOG.warn("The committed offsets request failed", e.getCause());

Review Comment:
   Same as above.



##########
clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java:
##########
@@ -3404,7 +3404,7 @@ public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String grou
                                                                    final ListConsumerGroupOffsetsOptions options) {
         SimpleAdminApiFuture<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> future =
                 ListConsumerGroupOffsetsHandler.newFuture(groupId);
-        ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), logContext);
+        ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), options.shouldRequireStable(), logContext);

Review Comment:
   Do you need to adapt/add a unit test for this change?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java:
##########
@@ -110,6 +112,44 @@ public static Map<MetricName, Metric> producerMetrics(final Collection<StreamsPr
         return result;
     }
 
+    /**
+     * @throws StreamsException if the consumer throws an exception
+     * @throws org.apache.kafka.common.errors.TimeoutException if the request times out
+     */
+    public static Map<TopicPartition, Long> fetchCommittedOffsets(final Set<TopicPartition> partitions,
+                                                                  final String groupId,
+                                                                  final Admin adminClient) {
+        if (partitions.isEmpty()) {
+            return Collections.emptyMap();
+        }
+
+        final Map<TopicPartition, Long> committedOffsets;
+        try {
+            // those which do not have a committed offset would default to 0
+            final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions();
+            options.topicPartitions(new ArrayList<>(partitions));
+            options.requireStable(true);
+            committedOffsets = adminClient.listConsumerGroupOffsets(groupId, options)
+                    .partitionsToOffsetAndMetadata().get().entrySet()
+                    .stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset()));
+        } catch (final InterruptedException e) {
+            LOG.warn("The committed offsets request failed due to interruption", e);
+            throw new StreamsException(String.format("Failed to retrieve end offsets for %s", partitions), e);

Review Comment:
   Why do we throw a `StreamsException` here? Currently, the state updater thread that calls this method gets interrupted during shutdown. So with this code a `StreamsException` might be thrown during shutdown of the state updater.
   I see two options here (there might be more):
   1. Move this method to the store changelog reader so that we ensure that only the state updater thread can call this method. Then we simply re-throw the `InterruptedException` since the state updater would shutdown anyways.
   2. Store the interruption in a flag, retry the remote call and if it succeeds, restore the interrupted state on the current thread. An example of that can be found [here](https://github.com/apache/kafka/blob/7f0af093a4d6f04ca50615cecf2f42c128996bd6/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java#L660).  



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java:
##########
@@ -110,6 +112,44 @@ public static Map<MetricName, Metric> producerMetrics(final Collection<StreamsPr
         return result;
     }
 
+    /**
+     * @throws StreamsException if the consumer throws an exception
+     * @throws org.apache.kafka.common.errors.TimeoutException if the request times out
+     */
+    public static Map<TopicPartition, Long> fetchCommittedOffsets(final Set<TopicPartition> partitions,
+                                                                  final String groupId,
+                                                                  final Admin adminClient) {
+        if (partitions.isEmpty()) {
+            return Collections.emptyMap();
+        }
+
+        final Map<TopicPartition, Long> committedOffsets;
+        try {
+            // those which do not have a committed offset would default to 0
+            final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions();
+            options.topicPartitions(new ArrayList<>(partitions));
+            options.requireStable(true);
+            committedOffsets = adminClient.listConsumerGroupOffsets(groupId, options)
+                    .partitionsToOffsetAndMetadata().get().entrySet()
+                    .stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset()));
+        } catch (final InterruptedException e) {
+            LOG.warn("The committed offsets request failed due to interruption", e);
+            throw new StreamsException(String.format("Failed to retrieve end offsets for %s", partitions), e);
+        } catch (final ExecutionException e) {
+            if (e.getCause() instanceof TimeoutException) {
+                final TimeoutException exception = (TimeoutException) e.getCause();
+                LOG.warn("The committed offsets request timed out, try increasing the consumer client's default.api.timeout.ms", exception);

Review Comment:
   Could we please use something like "Retrieving the committed offset timed out"? I get always confused about "committed offsets" and "commit offsets". I think making it clearer improves debugability. 



##########
clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java:
##########
@@ -44,10 +45,21 @@ public ListConsumerGroupOffsetsOptions topicPartitions(List<TopicPartition> topi
         return this;
     }
 
+    /**
+     * Sets an optional requireStable flag.
+     */
+    public void requireStable(final boolean requireStable) {
+        this.requireStable = requireStable;
+    }
+
     /**
      * Returns a list of topic partitions to add as part of the result.
      */
     public List<TopicPartition> topicPartitions() {
         return topicPartitions;

Review Comment:
   Sorry for commenting on existing code. Should we return a immutable list here?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java:
##########
@@ -110,6 +112,44 @@ public static Map<MetricName, Metric> producerMetrics(final Collection<StreamsPr
         return result;
     }
 
+    /**
+     * @throws StreamsException if the consumer throws an exception
+     * @throws org.apache.kafka.common.errors.TimeoutException if the request times out
+     */
+    public static Map<TopicPartition, Long> fetchCommittedOffsets(final Set<TopicPartition> partitions,

Review Comment:
   If this method will remain here (see my other comment about `InterruptedException`), it needs unit tests.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java:
##########
@@ -697,7 +693,7 @@ private Map<TopicPartition, Long> committedOffsetForChangelogs(final Map<TaskId,
                                                                    final Set<TopicPartition> partitions) {
         final Map<TopicPartition, Long> committedOffsets;
         try {
-            committedOffsets = fetchCommittedOffsets(partitions, mainConsumer);

Review Comment:
   Commenting here because I cannot comment on `endOffsetForChangelogs()`. Method `endOffsetForChangelogs()` should re-throw the `InterruptedException` since interruption is used for the shutdown of the state updater. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org