lianetm commented on code in PR #20324: URL: https://github.com/apache/kafka/pull/20324#discussion_r2550854575
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PositionsValidator.java: ########## @@ -0,0 +1,157 @@ +/* + * 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.consumer.Consumer; +import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent; +import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +import org.slf4j.Logger; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static java.util.Objects.requireNonNull; + +/** + * As named, this class validates positions in the {@link SubscriptionState} based on current {@link ConsumerMetadata} + * version. It maintains just enough shared state to determine when it can avoid costly inter-thread communication + * in the {@link Consumer#poll(Duration)} method. + * + * <p/> + * + * Callers from the application thread should not mutate any of the state contained within this class. + * It should be considered as <em>read-only</em>, and only the background thread should mutate the state. + */ +public class PositionsValidator { + + private final Logger log; + private final Time time; + private final ConsumerMetadata metadata; + private final SubscriptionState subscriptions; + + /** + * Exception that occurred while validating positions, that will be propagated on the next + * call to validate positions. This could be an error received in the + * OffsetsForLeaderEpoch response, or a LogTruncationException detected when using a + * successful response to validate the positions. It will be cleared when thrown. + */ + private final AtomicReference<RuntimeException> cachedValidatePositionsException = new AtomicReference<>(); + + private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1); + + public PositionsValidator(LogContext logContext, + Time time, + ConsumerMetadata metadata, + SubscriptionState subscriptions) { + this.log = requireNonNull(logContext).logger(getClass()); + this.time = requireNonNull(time); + this.metadata = requireNonNull(metadata); + this.subscriptions = requireNonNull(subscriptions); + } + + /** + * This method is called by the background thread in response to {@link AsyncPollEvent} and + * {@link CheckAndUpdatePositionsEvent}. + */ + Map<TopicPartition, SubscriptionState.FetchPosition> getPartitionsToValidate(ApiVersions apiVersions) { + maybeThrowError(); + + // Validate each partition against the current leader and epoch + // If we see a new metadata version, check all partitions + validatePositionsOnMetadataChange(apiVersions); + + // Collect positions needing validation, with backoff + return subscriptions.partitionsNeedingValidation(time.milliseconds()); + } + + /** + * If we have seen new metadata (as tracked by {@link org.apache.kafka.clients.Metadata#updateVersion()}), then + * we should check that all the assignments have a valid position. + */ + void validatePositionsOnMetadataChange(ApiVersions apiVersions) { + int newMetadataUpdateVersion = metadata.updateVersion(); + if (metadataUpdateVersion.getAndSet(newMetadataUpdateVersion) != newMetadataUpdateVersion) { + subscriptions.assignedPartitions().forEach(topicPartition -> { + ConsumerMetadata.LeaderAndEpoch leaderAndEpoch = metadata.currentLeader(topicPartition); + subscriptions.maybeValidatePositionForCurrentLeader(apiVersions, topicPartition, leaderAndEpoch); + }); + } + } + + void maybeSetError(RuntimeException e) { + if (!cachedValidatePositionsException.compareAndSet(null, e)) { + log.error("Discarding error validating positions because another error is pending", e); + } + } + + void maybeThrowError() { + RuntimeException exception = cachedValidatePositionsException.getAndSet(null); + if (exception != null) + throw exception; + } + + /** + * This method is used by {@code AsyncKafkaConsumer} to determine if it can skip the step of validating + * positions as this is in the critical path for the {@link Consumer#poll(Duration)}. If the application thread + * can safely and accurately determine that it doesn't need to perform the + * {@link OffsetsRequestManager#updateFetchPositions(long)} call, a big performance savings can be realized. + * + * <p/> + * + * This method performs similar checks to the start of {@link OffsetsRequestManager#updateFetchPositions(long)}: + * + * <ol> + * <li> + * Checks that there are no positions in the {@link SubscriptionState.FetchStates#AWAIT_VALIDATION} + * state ({@link OffsetFetcherUtils#getPartitionsToValidate()}) + * </li> + * <li> + * Checks that all positions are in the {@link SubscriptionState.FetchStates#FETCHING} state + * ({@link SubscriptionState#hasAllFetchPositions()}) + * </li> + * </ol> + * + * If any checks fail, this method will return {@code false}, otherwise, it will return {@code true}, which + * signals to the application thread that the position validation step can be skipped. + * + * @return true if all checks pass, false if any checks fail + */ + boolean canSkipUpdateFetchPositions() { + maybeThrowError(); + + if (metadataUpdateVersion.get() != metadata.updateVersion()) { + return false; + } + + // In cases of metadata updates, getPartitionsToValidate() will review the partitions and + // determine which, if any, need to be validated. If any partitions require validation, the + // update fetch positions step can't be skipped. + if (subscriptions.hasPartitionsNeedingValidation(time.milliseconds())) { Review Comment: yes, agree, but that situation happens no matter what checks we do here (ex. would happen also with the previous check of `hasPartitionsNeedingValidation`), and was possible also before this PR actually, when we had a blocking poll with `CheckAndUpdatePositions` (metadata could change right after the app thread regained control after CheckAndUpdatePositions, but the app thread would still collectFetch, right?) The good thing is that the `collectFetch` does check the basics for correctness before returning data from the buffer (partition still assigned, with valid position). So if the metadata did change in the background and the position was marked as invalid the data won't be returned, if not, I expect it's safe to return (even though there may be a metadata change in-progress let's say). -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
