junrao commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1367537583
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -639,56 +835,175 @@ public void assign(Collection<TopicPartition> partitions) { throw new IllegalArgumentException("Topic partitions to assign to cannot have null or empty topic"); } - // TODO: implementation of refactored Fetcher will be included in forthcoming commits. - // fetcher.clearBufferedDataForUnassignedPartitions(partitions); + // Clear the buffered data which are not a part of newly assigned topics + final Set<TopicPartition> currentTopicPartitions = new HashSet<>(); + + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (partitions.contains(tp)) + currentTopicPartitions.add(tp); + } + + fetchBuffer.retainAll(currentTopicPartitions); // assignment change event will trigger autocommit if it is configured and the group id is specified. This is // to make sure offsets of topic partitions the consumer is unsubscribing from are committed since there will - // be no following rebalance - eventHandler.add(new AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), time.milliseconds())); + // be no following rebalance. + // + // See the ApplicationEventProcessor.process() method that handles this event for more detail. + applicationEventHandler.add(new AssignmentChangeApplicationEvent(subscriptions.allConsumed(), time.milliseconds())); log.info("Assigned to partition(s): {}", join(partitions, ", ")); - if (this.subscriptions.assignFromUser(new HashSet<>(partitions))) - eventHandler.add(new NewTopicsMetadataUpdateRequestEvent()); + if (subscriptions.assignFromUser(new HashSet<>(partitions))) + applicationEventHandler.add(new NewTopicsMetadataUpdateRequestEvent()); } @Override - public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) { - throw new KafkaException("method not implemented"); + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + maybeThrowInvalidGroupIdException(); + if (pattern == null || pattern.toString().isEmpty()) + throw new IllegalArgumentException("Topic pattern to subscribe to cannot be " + (pattern == null ? + "null" : "empty")); + + throwIfNoAssignorsConfigured(); + log.info("Subscribed to pattern: '{}'", pattern); + subscriptions.subscribe(pattern, listener); + updatePatternSubscription(metadata.fetch()); + metadata.requestUpdateForNewTopics(); + } + + /** + * TODO: remove this when we implement the KIP-848 protocol. + * + * <p> + * The contents of this method are shamelessly stolen from + * {@link ConsumerCoordinator#updatePatternSubscription(Cluster)} and are used here because we won't have access + * to a {@link ConsumerCoordinator} in this code. Perhaps it could be moved to a ConsumerUtils class? + * + * @param cluster Cluster from which we get the topics + */ + private void updatePatternSubscription(Cluster cluster) { + final Set<String> topicsToSubscribe = cluster.topics().stream() + .filter(subscriptions::matchesSubscribedPattern) + .collect(Collectors.toSet()); + if (subscriptions.subscribeFromPattern(topicsToSubscribe)) + metadata.requestUpdateForNewTopics(); } @Override public void subscribe(Pattern pattern) { - throw new KafkaException("method not implemented"); + subscribe(pattern, new NoOpConsumerRebalanceListener()); } @Override public void unsubscribe() { - throw new KafkaException("method not implemented"); + fetchBuffer.retainAll(Collections.emptySet()); + subscriptions.unsubscribe(); } @Override @Deprecated - public ConsumerRecords<K, V> poll(long timeout) { - throw new KafkaException("method not implemented"); + public ConsumerRecords<K, V> poll(final long timeoutMs) { + return poll(Duration.ofMillis(timeoutMs)); } // Visible for testing WakeupTrigger wakeupTrigger() { return wakeupTrigger; } - private static <K, V> ClusterResourceListeners configureClusterResourceListeners( - final Deserializer<K> keyDeserializer, - final Deserializer<V> valueDeserializer, - final List<?>... candidateLists) { - ClusterResourceListeners clusterResourceListeners = new ClusterResourceListeners(); - for (List<?> candidateList: candidateLists) - clusterResourceListeners.maybeAddAll(candidateList); + private Fetch<K, V> pollForFetches(Timer timer) { + long pollTimeout = timer.remainingMs(); + + // if data is available already, return it immediately + final Fetch<K, V> fetch = collectFetch(); + if (!fetch.isEmpty()) { + return fetch; + } + + // We do not want to be stuck blocking in poll if we are missing some positions + // since the offset lookup may be backing off after a failure + + // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call + // updateAssignmentMetadataIfNeeded before this method. + if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > retryBackoffMs) { + pollTimeout = retryBackoffMs; + } + + log.trace("Polling for fetches with timeout {}", pollTimeout); + + Timer pollTimer = time.timer(pollTimeout); - clusterResourceListeners.maybeAdd(keyDeserializer); - clusterResourceListeners.maybeAdd(valueDeserializer); - return clusterResourceListeners; + // Wait a bit for some fetched data to arrive, as there may not be anything immediately available. Note the + // use of a shorter, dedicated "pollTimer" here which updates "timer" so that calling method (poll) will + // correctly handle the overall timeout. + try { + fetchBuffer.awaitNotEmpty(pollTimer); + } catch (InterruptException e) { + log.trace("Timeout during fetch", e); + } finally { + timer.update(pollTimer.currentTimeMs()); + } + + return collectFetch(); Review Comment: Could we just return empty here instead of calling collectFetch() again since the caller is in a loop and can call this method to collect fetch again? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -639,56 +835,175 @@ public void assign(Collection<TopicPartition> partitions) { throw new IllegalArgumentException("Topic partitions to assign to cannot have null or empty topic"); } - // TODO: implementation of refactored Fetcher will be included in forthcoming commits. - // fetcher.clearBufferedDataForUnassignedPartitions(partitions); + // Clear the buffered data which are not a part of newly assigned topics + final Set<TopicPartition> currentTopicPartitions = new HashSet<>(); + + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (partitions.contains(tp)) + currentTopicPartitions.add(tp); + } + + fetchBuffer.retainAll(currentTopicPartitions); // assignment change event will trigger autocommit if it is configured and the group id is specified. This is // to make sure offsets of topic partitions the consumer is unsubscribing from are committed since there will - // be no following rebalance - eventHandler.add(new AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), time.milliseconds())); + // be no following rebalance. + // + // See the ApplicationEventProcessor.process() method that handles this event for more detail. + applicationEventHandler.add(new AssignmentChangeApplicationEvent(subscriptions.allConsumed(), time.milliseconds())); log.info("Assigned to partition(s): {}", join(partitions, ", ")); - if (this.subscriptions.assignFromUser(new HashSet<>(partitions))) - eventHandler.add(new NewTopicsMetadataUpdateRequestEvent()); + if (subscriptions.assignFromUser(new HashSet<>(partitions))) + applicationEventHandler.add(new NewTopicsMetadataUpdateRequestEvent()); } @Override - public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) { - throw new KafkaException("method not implemented"); + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { + maybeThrowInvalidGroupIdException(); + if (pattern == null || pattern.toString().isEmpty()) + throw new IllegalArgumentException("Topic pattern to subscribe to cannot be " + (pattern == null ? + "null" : "empty")); + + throwIfNoAssignorsConfigured(); + log.info("Subscribed to pattern: '{}'", pattern); + subscriptions.subscribe(pattern, listener); + updatePatternSubscription(metadata.fetch()); + metadata.requestUpdateForNewTopics(); + } + + /** + * TODO: remove this when we implement the KIP-848 protocol. + * + * <p> + * The contents of this method are shamelessly stolen from + * {@link ConsumerCoordinator#updatePatternSubscription(Cluster)} and are used here because we won't have access + * to a {@link ConsumerCoordinator} in this code. Perhaps it could be moved to a ConsumerUtils class? + * + * @param cluster Cluster from which we get the topics + */ + private void updatePatternSubscription(Cluster cluster) { + final Set<String> topicsToSubscribe = cluster.topics().stream() + .filter(subscriptions::matchesSubscribedPattern) + .collect(Collectors.toSet()); + if (subscriptions.subscribeFromPattern(topicsToSubscribe)) + metadata.requestUpdateForNewTopics(); } @Override public void subscribe(Pattern pattern) { - throw new KafkaException("method not implemented"); + subscribe(pattern, new NoOpConsumerRebalanceListener()); } @Override public void unsubscribe() { - throw new KafkaException("method not implemented"); + fetchBuffer.retainAll(Collections.emptySet()); + subscriptions.unsubscribe(); } @Override @Deprecated - public ConsumerRecords<K, V> poll(long timeout) { - throw new KafkaException("method not implemented"); + public ConsumerRecords<K, V> poll(final long timeoutMs) { + return poll(Duration.ofMillis(timeoutMs)); } // Visible for testing WakeupTrigger wakeupTrigger() { return wakeupTrigger; } - private static <K, V> ClusterResourceListeners configureClusterResourceListeners( - final Deserializer<K> keyDeserializer, - final Deserializer<V> valueDeserializer, - final List<?>... candidateLists) { - ClusterResourceListeners clusterResourceListeners = new ClusterResourceListeners(); - for (List<?> candidateList: candidateLists) - clusterResourceListeners.maybeAddAll(candidateList); + private Fetch<K, V> pollForFetches(Timer timer) { + long pollTimeout = timer.remainingMs(); + + // if data is available already, return it immediately + final Fetch<K, V> fetch = collectFetch(); + if (!fetch.isEmpty()) { + return fetch; + } + + // We do not want to be stuck blocking in poll if we are missing some positions + // since the offset lookup may be backing off after a failure + + // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call + // updateAssignmentMetadataIfNeeded before this method. + if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > retryBackoffMs) { + pollTimeout = retryBackoffMs; + } + + log.trace("Polling for fetches with timeout {}", pollTimeout); + + Timer pollTimer = time.timer(pollTimeout); - clusterResourceListeners.maybeAdd(keyDeserializer); - clusterResourceListeners.maybeAdd(valueDeserializer); - return clusterResourceListeners; + // Wait a bit for some fetched data to arrive, as there may not be anything immediately available. Note the + // use of a shorter, dedicated "pollTimer" here which updates "timer" so that calling method (poll) will + // correctly handle the overall timeout. + try { + fetchBuffer.awaitNotEmpty(pollTimer); + } catch (InterruptException e) { + log.trace("Timeout during fetch", e); + } finally { + timer.update(pollTimer.currentTimeMs()); + } + + return collectFetch(); + } + + /** + * Perform the "{@link FetchCollector#collectFetch(FetchBuffer) fetch collection}" step by reading raw data out + * of the {@link #fetchBuffer}, converting it to a well-formed {@link CompletedFetch}, validating that it and + * the internal {@link SubscriptionState state} are correct, and then converting it all into a {@link Fetch} + * for returning. + * + * <p/> + * + * This method will {@link ApplicationEventHandler#wakeupNetworkThread() wake up} the {@link ConsumerNetworkThread} before + * retuning. This is done as an optimization so that the <em>next round of data can be pre-fetched</em>. + */ + private Fetch<K, V> collectFetch() { + final Fetch<K, V> fetch = fetchCollector.collectFetch(fetchBuffer); + + // Notify the network thread to wake up and start the next round of fetching. + applicationEventHandler.wakeupNetworkThread(); + + return fetch; + } + /** + * Set the fetch position to the committed position (if there is one) + * or reset it using the offset reset policy the user has configured. + * + * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details + * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is + * defined + * @return true iff the operation completed without timing out + */ + private boolean updateFetchPositions(final Timer timer) { + // Validate positions using the partition leader end offsets, to detect if any partition + // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch + // request, retrieve the partition end offsets, and validate the current position against it. + applicationEventHandler.add(new ValidatePositionsApplicationEvent()); + + cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); + if (cachedSubscriptionHasAllFetchPositions) return true; + + // Reset positions using committed offsets retrieved from the group coordinator, for any + // partitions which do not have a valid position and are not awaiting reset. This will + // trigger an OffsetFetch request and update positions with the offsets retrieved. This + // will only do a coordinator lookup if there are partitions which have missing + // positions, so a consumer with manually assigned partitions can avoid a coordinator + // dependence by always ensuring that assigned partitions have an initial position. + if (isCommittedOffsetsManagementEnabled() && !refreshCommittedOffsetsIfNeeded(timer)) + return false; Review Comment: Would it be clearer to rename `refreshCommittedOffsetsIfNeeded` to `initWithCommittedOffsetsIfNeeded`? -- 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