kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348083698
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -636,42 +857,148 @@ public void assign(Collection<TopicPartition>
partitions) {
}
@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().equals(""))
+ throw new IllegalArgumentException("Topic pattern to subscribe to
cannot be " + (pattern == null ?
+ "null" : "empty"));
+
+ throwIfNoAssignorsConfigured();
+ log.info("Subscribed to pattern: '{}'", pattern);
+ this.subscriptions.subscribe(pattern, listener);
+ this.updatePatternSubscription(metadata.fetch());
+ this.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());
+ this.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 void sendFetches() {
+ FetchEvent event = new FetchEvent();
+ eventHandler.add(event);
+
+ event.future().whenComplete((completedFetches, error) -> {
+ if (completedFetches != null && !completedFetches.isEmpty()) {
+ fetchBuffer.addAll(completedFetches);
+ }
+ });
+ }
+
+ /**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+ private Fetch<K, V> pollForFetches(Timer timer) {
+ long pollTimeout = timer.remainingMs();
+
+ // if data is available already, return it immediately
+ final Fetch<K, V> fetch = fetchCollector.collectFetch(fetchBuffer);
+ if (!fetch.isEmpty()) {
+ return fetch;
+ }
+
+ // send any new fetches (won't resend pending fetches)
+ sendFetches();
+
+ // 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);
+
+ // Attempt to fetch any data. It's OK if we time out here; it's a best
case effort. The
+ // data may not be immediately available, but the calling method
(poll) will correctly
+ // handle the overall timeout.
+ try {
+ Queue<CompletedFetch> completedFetches =
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+ if (completedFetches != null && !completedFetches.isEmpty()) {
+ fetchBuffer.addAll(completedFetches);
+ }
+ } catch (TimeoutException e) {
+ log.trace("Timeout during fetch", e);
+ } finally {
+ timer.update(pollTimer.currentTimeMs());
+ }
- clusterResourceListeners.maybeAdd(keyDeserializer);
- clusterResourceListeners.maybeAdd(valueDeserializer);
- return clusterResourceListeners;
+ return fetchCollector.collectFetch(fetchBuffer);
+ }
+
+ /**
+ * 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) {
+ // If any partitions have been truncated due to a leader change, we
need to validate the offsets
+ eventHandler.add(new ValidatePositionsApplicationEvent());
Review Comment:
Yes, it appears that it will continue to attempt to fetch from the old
leader.
When a fetch response is received from the broker and we notice that it has
an error, we call `FetchCollector.handleInitializeErrors()` to deal with the
different error conditions. When it notices that the error is
`FENCED_LEADER_EPOCH`, it will execute these two statements (from the
`FetchUtils.requestMetadataUpdate()` method):
```java
metadata.requestUpdate(false);
subscriptions.clearPreferredReadReplica(topicPartition);
```
That's all it does. The logic doesn't update the `FetchState` for that
partition. It doesn't clear out the leader epoch. Nothing.
The next time the user calls the `Consumer.poll()` method, the
`Fetcher`/`FetchRequestManager` will determine for which partitions we should
issue `FETCH` RPCs. The first step is to call the
`SubscriptionState.fetchablePartitions()` method which checks each partition:
```java
public synchronized List<TopicPartition>
fetchablePartitions(Predicate<TopicPartition> isAvailable) {
// Since this is in the hot-path for fetching, we do this instead of
using java.util.stream API
List<TopicPartition> result = new ArrayList<>();
assignment.forEach((topicPartition, topicPartitionState) -> {
// Cheap check is first to avoid evaluating the predicate if possible
if (topicPartitionState.isFetchable() &&
isAvailable.test(topicPartition)) {
result.add(topicPartition);
}
});
return result;
}
```
Because we didn't change anything in the underlying state of the partition
in `SubscriptionState` previously, when the `topicPartitionState.isFetchable()`
method is invoked, it returns `true`. Thus it is included in the list for which
to fetch, and we will likely hit the same error.
--
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]