You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "guozhangwang (via GitHub)" <gi...@apache.org> on 2023/02/15 18:53:01 UTC

[GitHub] [kafka] guozhangwang commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

guozhangwang commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1107569600


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MetadataFetcher.java:
##########
@@ -0,0 +1,805 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.clients.StaleMetadataException;
+import org.apache.kafka.clients.consumer.LogTruncationException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
+import org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+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;
+
+public class MetadataFetcher {
+
+    private final Logger log;
+    private final ConsumerMetadata metadata;
+    private final SubscriptionState subscriptions;
+    private final ConsumerNetworkClient client;
+    private final Time time;
+    private final long retryBackoffMs;
+    private final long requestTimeoutMs;
+    private final IsolationLevel isolationLevel;
+    private final AtomicReference<RuntimeException> cachedListOffsetsException = new AtomicReference<>();
+    private final AtomicReference<RuntimeException> cachedOffsetForLeaderException = new AtomicReference<>();
+    private final OffsetsForLeaderEpochClient offsetsForLeaderEpochClient;
+    private final ApiVersions apiVersions;
+    private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1);
+
+    public MetadataFetcher(LogContext logContext,
+                           ConsumerNetworkClient client,
+                           ConsumerMetadata metadata,
+                           SubscriptionState subscriptions,
+                           Time time,
+                           long retryBackoffMs,
+                           long requestTimeoutMs,
+                           IsolationLevel isolationLevel,
+                           ApiVersions apiVersions) {
+        this.log = logContext.logger(getClass());
+        this.time = time;
+        this.client = client;
+        this.metadata = metadata;
+        this.subscriptions = subscriptions;
+        this.retryBackoffMs = retryBackoffMs;
+        this.requestTimeoutMs = requestTimeoutMs;
+        this.isolationLevel = isolationLevel;
+        this.apiVersions = apiVersions;
+        this.offsetsForLeaderEpochClient = new OffsetsForLeaderEpochClient(client, logContext);
+    }
+
+    /**
+     * Represents data about an offset returned by a broker.
+     */
+    static class ListOffsetData {
+        final long offset;
+        final Long timestamp; //  null if the broker does not support returning timestamps
+        final Optional<Integer> leaderEpoch; // empty if the leader epoch is not known
+
+        ListOffsetData(long offset, Long timestamp, Optional<Integer> leaderEpoch) {
+            this.offset = offset;
+            this.timestamp = timestamp;
+            this.leaderEpoch = leaderEpoch;
+        }
+    }
+
+    private Long offsetResetStrategyTimestamp(final TopicPartition partition) {
+        OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
+        if (strategy == OffsetResetStrategy.EARLIEST)
+            return ListOffsetsRequest.EARLIEST_TIMESTAMP;
+        else if (strategy == OffsetResetStrategy.LATEST)
+            return ListOffsetsRequest.LATEST_TIMESTAMP;
+        else
+            return null;
+    }
+
+    private OffsetResetStrategy timestampToOffsetResetStrategy(long timestamp) {
+        if (timestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP)
+            return OffsetResetStrategy.EARLIEST;
+        else if (timestamp == ListOffsetsRequest.LATEST_TIMESTAMP)
+            return OffsetResetStrategy.LATEST;
+        else
+            return null;
+    }
+
+    /**
+     * Get topic metadata for all topics in the cluster
+     * @param timer Timer bounding how long this method can block
+     * @return The map of topics with their partition information
+     */
+    public Map<String, List<PartitionInfo>> getAllTopicMetadata(Timer timer) {
+        return getTopicMetadata(MetadataRequest.Builder.allTopics(), timer);
+    }
+
+    /**
+     * Get metadata for all topics present in Kafka cluster
+     *
+     * @param request The MetadataRequest to send
+     * @param timer Timer bounding how long this method can block
+     * @return The map of topics with their partition information
+     */
+    public Map<String, List<PartitionInfo>> getTopicMetadata(MetadataRequest.Builder request, Timer timer) {
+        // Save the round trip if no topics are requested.
+        if (!request.isAllTopics() && request.emptyTopicList())
+            return Collections.emptyMap();
+
+        do {
+            RequestFuture<ClientResponse> future = sendMetadataRequest(request);
+            client.poll(future, timer);
+
+            if (future.failed() && !future.isRetriable())
+                throw future.exception();
+
+            if (future.succeeded()) {
+                MetadataResponse response = (MetadataResponse) future.value().responseBody();
+                Cluster cluster = response.buildCluster();
+
+                Set<String> unauthorizedTopics = cluster.unauthorizedTopics();
+                if (!unauthorizedTopics.isEmpty())
+                    throw new TopicAuthorizationException(unauthorizedTopics);
+
+                boolean shouldRetry = false;
+                Map<String, Errors> errors = response.errors();
+                if (!errors.isEmpty()) {
+                    // if there were errors, we need to check whether they were fatal or whether
+                    // we should just retry
+
+                    log.debug("Topic metadata fetch included errors: {}", errors);
+
+                    for (Map.Entry<String, Errors> errorEntry : errors.entrySet()) {
+                        String topic = errorEntry.getKey();
+                        Errors error = errorEntry.getValue();
+
+                        if (error == Errors.INVALID_TOPIC_EXCEPTION)
+                            throw new InvalidTopicException("Topic '" + topic + "' is invalid");
+                        else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION)
+                            // if a requested topic is unknown, we just continue and let it be absent
+                            // in the returned map
+                            continue;
+                        else if (error.exception() instanceof RetriableException)
+                            shouldRetry = true;
+                        else
+                            throw new KafkaException("Unexpected error fetching metadata for topic " + topic,
+                                    error.exception());
+                    }
+                }
+
+                if (!shouldRetry) {
+                    HashMap<String, List<PartitionInfo>> topicsPartitionInfos = new HashMap<>();
+                    for (String topic : cluster.topics())
+                        topicsPartitionInfos.put(topic, cluster.partitionsForTopic(topic));
+                    return topicsPartitionInfos;
+                }
+            }
+
+            timer.sleep(retryBackoffMs);
+        } while (timer.notExpired());
+
+        throw new TimeoutException("Timeout expired while fetching topic metadata");
+    }
+
+    /**
+     * Send Metadata Request to least loaded node in Kafka cluster asynchronously
+     * @return A future that indicates result of sent metadata request
+     */
+    private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builder request) {
+        final Node node = client.leastLoadedNode();
+        if (node == null)
+            return RequestFuture.noBrokersAvailable();
+        else
+            return client.send(node, request);
+    }
+
+    /**
+     * Reset offsets for all assigned partitions that require it.
+     *
+     * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
+     *   and one or more partitions aren't awaiting a seekToBeginning() or seekToEnd().
+     */
+    public void resetOffsetsIfNeeded() {
+        // Raise exception from previous offset fetch if there is one
+        RuntimeException exception = cachedListOffsetsException.getAndSet(null);
+        if (exception != null)
+            throw exception;
+
+        Set<TopicPartition> partitions = subscriptions.partitionsNeedingReset(time.milliseconds());
+        if (partitions.isEmpty())
+            return;
+
+        final Map<TopicPartition, Long> offsetResetTimestamps = new HashMap<>();
+        for (final TopicPartition partition : partitions) {
+            Long timestamp = offsetResetStrategyTimestamp(partition);
+            if (timestamp != null)
+                offsetResetTimestamps.put(partition, timestamp);
+        }
+
+        resetOffsetsAsync(offsetResetTimestamps);
+    }
+
+    /**
+     * Validate offsets for all assigned partitions for which a leader change has been detected.
+     */
+    public void validateOffsetsIfNeeded() {

Review Comment:
   I'd recommend renaming all functions like `validateOffsetsIfNeeded` / `resetOffsetsIfNeeded` here by replacing `Offset` to `Position`, since 1) we do not only just validate / reset offset value, but the whole position including epoch etc as well, 2) this is also to distinguish from the other `offsets` like the ones from fetch responses as well moving on.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MetadataFetcher.java:
##########
@@ -0,0 +1,805 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.clients.StaleMetadataException;
+import org.apache.kafka.clients.consumer.LogTruncationException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
+import org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+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;
+
+public class MetadataFetcher {

Review Comment:
   What about calling it `TopicMetadataFetcher` to distinguish with other metadatas?



-- 
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