From 2b233bfa5f35bf237effe8c5202fd2b80c601943 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 24 Oct 2023 13:03:05 -0700 Subject: [PATCH] KAFKA-14274 [6, 7]: Introduction of fetch request manager (#14406) Changes: 1. Introduces FetchRequestManager that implements the RequestManager API for fetching messages from brokers. Unlike Fetcher, record decompression and deserialization is performed on the application thread inside CompletedFetch. 2. Restructured the code so that objects owned by the background thread are not instantiated until the background thread runs (via Supplier) to ensure that there are no references available to the application thread. 3. Ensuring resources are properly using Closeable and using IdempotentCloser to ensure they're only closed once. 4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in the way the objects were built up for tests. Reviewers: Philip Nee , Lianet Magrans , Jun Rao --- checkstyle/import-control.xml | 4 + checkstyle/suppressions.xml | 12 +- .../kafka/clients/consumer/KafkaConsumer.java | 11 +- .../kafka/clients/consumer/MockConsumer.java | 2 +- .../consumer/internals/AbstractFetch.java | 169 +- ...rEventHandler.java => CachedSupplier.java} | 29 +- .../internals/CommitRequestManager.java | 17 +- .../consumer/internals/CompletedFetch.java | 28 +- .../internals/ConsumerCoordinator.java | 10 +- .../internals/ConsumerNetworkThread.java | 268 ++ .../consumer/internals/ConsumerUtils.java | 18 +- .../internals/CoordinatorRequestManager.java | 28 +- .../internals/DefaultBackgroundThread.java | 293 -- .../internals/DefaultEventHandler.java | 151 - .../consumer/internals/Deserializers.java | 8 + .../consumer/internals/FetchBuffer.java | 169 +- .../consumer/internals/FetchCollector.java | 11 +- .../consumer/internals/FetchConfig.java | 84 +- .../internals/FetchRequestManager.java | 117 + .../clients/consumer/internals/Fetcher.java | 130 +- .../internals/HeartbeatRequestManager.java | 40 +- .../internals/NetworkClientDelegate.java | 123 +- .../internals/OffsetFetcherUtils.java | 3 +- .../internals/OffsetsRequestManager.java | 25 +- .../internals/PrototypeAsyncConsumer.java | 759 +++- .../consumer/internals/RequestManager.java | 37 + .../consumer/internals/RequestManagers.java | 132 +- .../TopicMetadataRequestManager.java | 6 +- .../internals/events/ApplicationEvent.java | 2 +- .../events/ApplicationEventHandler.java | 115 + .../events/ApplicationEventProcessor.java | 150 +- .../internals/events/BackgroundEvent.java | 6 +- .../events/BackgroundEventHandler.java | 52 + .../events/BackgroundEventProcessor.java | 75 + .../events/CompletableApplicationEvent.java | 2 +- .../internals/events/CompletableEvent.java | 25 + .../events/ErrorBackgroundEvent.java | 10 +- .../internals/events/EventHandler.java | 65 - .../internals/events/EventProcessor.java | 137 + .../events/NoopApplicationEvent.java | 62 - .../internals/events/NoopBackgroundEvent.java | 62 - .../clients/consumer/KafkaConsumerTest.java | 4 +- .../internals/CommitRequestManagerTest.java | 3 +- .../internals/CompletedFetchTest.java | 96 +- .../internals/ConsumerCoordinatorTest.java | 22 +- .../internals/ConsumerNetworkThreadTest.java | 269 ++ .../internals/ConsumerTestBuilder.java | 370 ++ .../CoordinatorRequestManagerTest.java | 21 +- .../DefaultBackgroundThreadTest.java | 436 -- .../internals/DefaultEventHandlerTest.java | 81 - .../internals/FetchCollectorTest.java | 10 +- .../consumer/internals/FetchConfigTest.java | 48 +- .../internals/FetchRequestManagerTest.java | 3512 +++++++++++++++++ .../consumer/internals/FetcherTest.java | 546 ++- .../HeartbeatRequestManagerTest.java | 201 +- .../consumer/internals/OffsetFetcherTest.java | 6 +- .../internals/OffsetsRequestManagerTest.java | 54 +- .../internals/PrototypeAsyncConsumerTest.java | 208 +- .../events/BackgroundEventHandlerTest.java | 141 + 59 files changed, 7023 insertions(+), 2452 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/{ErrorEventHandler.java => CachedSupplier.java} (61%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopApplicationEvent.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopBackgroundEvent.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThreadTest.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 46774912418..e8f8bd56547 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -205,6 +205,10 @@ + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index bd54231ffe5..e1e746c755a 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -61,6 +61,8 @@ files="AbstractRequest.java"/> + @@ -68,7 +70,7 @@ + files="(KafkaConsumer|PrototypeAsyncConsumer|ConsumerCoordinator).java"/> + files="(KafkaConsumer|PrototypeAsyncConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaRaftClient|KafkaRaftClientTest).java"/> @@ -108,10 +110,10 @@ + files="(Sender|Fetcher|FetchRequestManager|OffsetFetcher|KafkaConsumer|PrototypeAsyncConsumer|Metrics|RequestResponse|TransactionManager|KafkaAdminClient|Message|KafkaProducer)Test.java"/> + files="(ConsumerCoordinator|KafkaConsumer|RequestResponse|Fetcher|FetchRequestManager|KafkaAdminClient|Message|KafkaProducer)Test.java"/> @@ -120,7 +122,7 @@ files="(OffsetFetcher|RequestResponse)Test.java"/> + files="RequestResponseTest.java|FetcherTest.java|FetchRequestManagerTest.java|KafkaAdminClientTest.java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index f8221df36ae..dd273c38c43 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -78,14 +78,13 @@ import java.util.regex.Pattern; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createConsumerNetworkClient; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchConfig; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel; import static org.apache.kafka.common.utils.Utils.closeQuietly; import static org.apache.kafka.common.utils.Utils.isBlank; import static org.apache.kafka.common.utils.Utils.join; @@ -570,7 +569,6 @@ import static org.apache.kafka.common.utils.Utils.swallow; public class KafkaConsumer implements Consumer { private static final long NO_CURRENT_THREAD = -1L; - static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000; static final String DEFAULT_REASON = "rebalance enforced by user"; // Visible for testing @@ -715,7 +713,8 @@ public class KafkaConsumer implements Consumer { this.metadata.bootstrap(addresses); FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics); - this.isolationLevel = configuredIsolationLevel(config); + FetchConfig fetchConfig = new FetchConfig(config); + this.isolationLevel = fetchConfig.isolationLevel; ApiVersions apiVersions = new ApiVersions(); this.client = createConsumerNetworkClient(config, @@ -753,13 +752,13 @@ public class KafkaConsumer implements Consumer { config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED), config.getString(ConsumerConfig.CLIENT_RACK_CONFIG)); } - FetchConfig fetchConfig = createFetchConfig(config, this.deserializers); this.fetcher = new Fetcher<>( logContext, this.client, this.metadata, this.subscriptions, fetchConfig, + this.deserializers, fetchMetricsManager, this.time); this.offsetFetcher = new OffsetFetcher(logContext, @@ -2483,7 +2482,7 @@ public class KafkaConsumer implements Consumer { // 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 (coordinator != null && !coordinator.refreshCommittedOffsetsIfNeeded(timer)) return false; + if (coordinator != null && !coordinator.initWithCommittedOffsetsIfNeeded(timer)) return false; // If there are partitions still needing a position and a reset policy is defined, // request reset using the default policy. If no reset strategy is defined and there diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 26c6dd3e94b..5f143c9a079 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -46,7 +46,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import static java.util.Collections.singleton; -import static org.apache.kafka.clients.consumer.KafkaConsumer.DEFAULT_CLOSE_TIMEOUT_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java index 4f579bb0e0f..ffca2cee35d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java @@ -18,10 +18,13 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.FetchSessionHandler; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -37,61 +40,73 @@ import org.slf4j.Logger; import org.slf4j.helpers.MessageFormatter; import java.io.Closeable; -import java.util.ArrayList; +import java.time.Duration; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.function.Predicate; +import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.internals.FetchUtils.requestMetadataUpdate; /** * {@code AbstractFetch} represents the basic state and logic for record fetching processing. - * @param Type for the message key - * @param Type for the message value */ -public abstract class AbstractFetch implements Closeable { +public abstract class AbstractFetch implements Closeable { private final Logger log; + private final IdempotentCloser idempotentCloser = new IdempotentCloser(); protected final LogContext logContext; - protected final ConsumerNetworkClient client; protected final ConsumerMetadata metadata; protected final SubscriptionState subscriptions; - protected final FetchConfig fetchConfig; + protected final FetchConfig fetchConfig; protected final Time time; protected final FetchMetricsManager metricsManager; protected final FetchBuffer fetchBuffer; protected final BufferSupplier decompressionBufferSupplier; protected final Set nodesWithPendingFetchRequests; - protected final IdempotentCloser idempotentCloser = new IdempotentCloser(); private final Map sessionHandlers; public AbstractFetch(final LogContext logContext, - final ConsumerNetworkClient client, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final FetchConfig fetchConfig, + final FetchConfig fetchConfig, + final FetchBuffer fetchBuffer, final FetchMetricsManager metricsManager, final Time time) { this.log = logContext.logger(AbstractFetch.class); this.logContext = logContext; - this.client = client; this.metadata = metadata; this.subscriptions = subscriptions; this.fetchConfig = fetchConfig; + this.fetchBuffer = fetchBuffer; this.decompressionBufferSupplier = BufferSupplier.create(); - this.fetchBuffer = new FetchBuffer(logContext); this.sessionHandlers = new HashMap<>(); this.nodesWithPendingFetchRequests = new HashSet<>(); this.metricsManager = metricsManager; this.time = time; } + /** + * Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in + * reconnect backoff window following the disconnect). + * + * @param node {@link Node} to check for availability + * @see NetworkClientUtils#isUnavailable(KafkaClient, Node, Time) + */ + protected abstract boolean isUnavailable(Node node); + + /** + * Checks for an authentication error on a given node and throws the exception if it exists. + * + * @param node {@link Node} to check for a previous {@link AuthenticationException}; if found it is thrown + * @see NetworkClientUtils#maybeThrowAuthFailure(KafkaClient, Node) + */ + protected abstract void maybeThrowAuthFailure(Node node); + /** * Return whether we have any completed fetches pending return to the user. This method is thread-safe. Has * visibility for testing. @@ -111,15 +126,15 @@ public abstract class AbstractFetch implements Closeable { } /** - * Implements the core logic for a successful fetch request/response. + * Implements the core logic for a successful fetch response. * * @param fetchTarget {@link Node} from which the fetch data was requested * @param data {@link FetchSessionHandler.FetchRequestData} that represents the session data * @param resp {@link ClientResponse} from which the {@link FetchResponse} will be retrieved */ - protected void handleFetchResponse(final Node fetchTarget, - final FetchSessionHandler.FetchRequestData data, - final ClientResponse resp) { + protected void handleFetchSuccess(final Node fetchTarget, + final FetchSessionHandler.FetchRequestData data, + final ClientResponse resp) { try { final FetchResponse response = (FetchResponse) resp.responseBody(); final FetchSessionHandler handler = sessionHandler(fetchTarget.id()); @@ -185,18 +200,20 @@ public abstract class AbstractFetch implements Closeable { metricsManager.recordLatency(resp.requestLatencyMs()); } finally { - log.debug("Removing pending request for node {}", fetchTarget); - nodesWithPendingFetchRequests.remove(fetchTarget.id()); + removePendingFetchRequest(fetchTarget, data.metadata().sessionId()); } } /** - * Implements the core logic for a failed fetch request/response. + * Implements the core logic for a failed fetch response. * * @param fetchTarget {@link Node} from which the fetch data was requested - * @param t {@link Throwable} representing the error that resulted in the failure + * @param data {@link FetchSessionHandler.FetchRequestData} from request + * @param t {@link Throwable} representing the error that resulted in the failure */ - protected void handleFetchResponse(final Node fetchTarget, final Throwable t) { + protected void handleFetchFailure(final Node fetchTarget, + final FetchSessionHandler.FetchRequestData data, + final Throwable t) { try { final FetchSessionHandler handler = sessionHandler(fetchTarget.id()); @@ -205,25 +222,32 @@ public abstract class AbstractFetch implements Closeable { handler.sessionTopicPartitions().forEach(subscriptions::clearPreferredReadReplica); } } finally { - log.debug("Removing pending request for node {}", fetchTarget); - nodesWithPendingFetchRequests.remove(fetchTarget.id()); + removePendingFetchRequest(fetchTarget, data.metadata().sessionId()); } } - protected void handleCloseFetchSessionResponse(final Node fetchTarget, - final FetchSessionHandler.FetchRequestData data) { + protected void handleCloseFetchSessionSuccess(final Node fetchTarget, + final FetchSessionHandler.FetchRequestData data, + final ClientResponse ignored) { int sessionId = data.metadata().sessionId(); + removePendingFetchRequest(fetchTarget, sessionId); log.debug("Successfully sent a close message for fetch session: {} to node: {}", sessionId, fetchTarget); } - public void handleCloseFetchSessionResponse(final Node fetchTarget, - final FetchSessionHandler.FetchRequestData data, - final Throwable t) { + public void handleCloseFetchSessionFailure(final Node fetchTarget, + final FetchSessionHandler.FetchRequestData data, + final Throwable t) { int sessionId = data.metadata().sessionId(); - log.debug("Unable to a close message for fetch session: {} to node: {}. " + + removePendingFetchRequest(fetchTarget, sessionId); + log.debug("Unable to send a close message for fetch session: {} to node: {}. " + "This may result in unnecessary fetch sessions at the broker.", sessionId, fetchTarget, t); } + private void removePendingFetchRequest(Node fetchTarget, int sessionId) { + log.debug("Removing pending request for fetch session: {} for node: {}", sessionId, fetchTarget); + nodesWithPendingFetchRequests.remove(fetchTarget.id()); + } + /** * Creates a new {@link FetchRequest fetch request} in preparation for sending to the Kafka cluster. * @@ -317,9 +341,9 @@ public abstract class AbstractFetch implements Closeable { } } - private Map prepareCloseFetchSessionRequests() { + protected Map prepareCloseFetchSessionRequests() { final Cluster cluster = metadata.fetch(); - Map fetchable = new LinkedHashMap<>(); + Map fetchable = new HashMap<>(); try { sessionHandlers.forEach((fetchTargetNodeId, sessionHandler) -> { @@ -330,7 +354,7 @@ public abstract class AbstractFetch implements Closeable { // skip sending the close request. final Node fetchTarget = cluster.nodeById(fetchTargetNodeId); - if (fetchTarget == null || client.isUnavailable(fetchTarget)) { + if (fetchTarget == null || isUnavailable(fetchTarget)) { log.debug("Skip sending close session request to broker {} since it is not reachable", fetchTarget); return; } @@ -341,11 +365,7 @@ public abstract class AbstractFetch implements Closeable { sessionHandlers.clear(); } - Map reqs = new LinkedHashMap<>(); - for (Map.Entry entry : fetchable.entrySet()) { - reqs.put(entry.getKey(), entry.getValue().build()); - } - return reqs; + return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build())); } /** @@ -356,7 +376,7 @@ public abstract class AbstractFetch implements Closeable { // Update metrics in case there was an assignment change metricsManager.maybeUpdateAssignment(subscriptions); - Map fetchable = new LinkedHashMap<>(); + Map fetchable = new HashMap<>(); long currentTimeMs = time.milliseconds(); Map topicIds = metadata.topicIds(); @@ -377,8 +397,8 @@ public abstract class AbstractFetch implements Closeable { // Use the preferred read replica if set, otherwise the partition's leader Node node = selectReadReplica(partition, leaderOpt.get(), currentTimeMs); - if (client.isUnavailable(node)) { - client.maybeThrowAuthFailure(node); + if (isUnavailable(node)) { + maybeThrowAuthFailure(node); // If we try to send during the reconnect backoff window, then the request is just // going to be failed anyway before being sent, so skip sending the request for now @@ -405,51 +425,7 @@ public abstract class AbstractFetch implements Closeable { } } - Map reqs = new LinkedHashMap<>(); - for (Map.Entry entry : fetchable.entrySet()) { - reqs.put(entry.getKey(), entry.getValue().build()); - } - return reqs; - } - - protected void maybeCloseFetchSessions(final Timer timer) { - final List> requestFutures = new ArrayList<>(); - Map fetchRequestMap = prepareCloseFetchSessionRequests(); - - for (Map.Entry entry : fetchRequestMap.entrySet()) { - final Node fetchTarget = entry.getKey(); - final FetchSessionHandler.FetchRequestData data = entry.getValue(); - final FetchRequest.Builder request = createFetchRequest(fetchTarget, data); - final RequestFuture responseFuture = client.send(fetchTarget, request); - - responseFuture.addListener(new RequestFutureListener() { - @Override - public void onSuccess(ClientResponse value) { - handleCloseFetchSessionResponse(fetchTarget, data); - } - - @Override - public void onFailure(RuntimeException e) { - handleCloseFetchSessionResponse(fetchTarget, data, e); - } - }); - - requestFutures.add(responseFuture); - } - - // Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until - // all requests have received a response. - while (timer.notExpired() && !requestFutures.stream().allMatch(RequestFuture::isDone)) { - client.poll(timer, null, true); - } - - if (!requestFutures.stream().allMatch(RequestFuture::isDone)) { - // we ran out of time before completing all futures. It is ok since we don't want to block the shutdown - // here. - log.debug("All requests couldn't be sent in the specific timeout period {}ms. " + - "This may result in unnecessary fetch sessions at the broker. Consider increasing the timeout passed for " + - "KafkaConsumer.close(Duration timeout)", timer.timeoutMs()); - } + return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build())); } // Visible for testing @@ -467,20 +443,29 @@ public abstract class AbstractFetch implements Closeable { // Visible for testing protected void closeInternal(Timer timer) { // we do not need to re-enable wake-ups since we are closing already - client.disableWakeups(); - maybeCloseFetchSessions(timer); Utils.closeQuietly(fetchBuffer, "fetchBuffer"); Utils.closeQuietly(decompressionBufferSupplier, "decompressionBufferSupplier"); } public void close(final Timer timer) { - idempotentCloser.close(() -> { - closeInternal(timer); - }); + idempotentCloser.close(() -> closeInternal(timer)); } @Override public void close() { - close(time.timer(0)); + close(time.timer(Duration.ZERO)); + } + + /** + * Defines the contract for handling fetch responses from brokers. + * @param Type of response, usually either {@link ClientResponse} or {@link Throwable} + */ + @FunctionalInterface + protected interface ResponseHandler { + + /** + * Handle the response from the given {@link Node target} + */ + void handle(Node target, FetchSessionHandler.FetchRequestData data, T response); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ErrorEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CachedSupplier.java similarity index 61% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/ErrorEventHandler.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/CachedSupplier.java index 31c18731219..dc891a2539b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ErrorEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CachedSupplier.java @@ -16,19 +16,28 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import java.util.function.Supplier; -import java.util.Queue; +/** + * Simple {@link Supplier} that caches the initial creation of the object and stores it for later calls + * to {@link #get()}. + * + *

+ * + * Note: this class is not thread safe! Use only in contexts which are designed/guaranteed to be + * single-threaded. + */ +public abstract class CachedSupplier implements Supplier { -public class ErrorEventHandler { - private final Queue backgroundEventQueue; + private T result; - public ErrorEventHandler(Queue backgroundEventQueue) { - this.backgroundEventQueue = backgroundEventQueue; - } + protected abstract T create(); + + @Override + public T get() { + if (result == null) + result = create(); - public void handle(Throwable e) { - backgroundEventQueue.add(new ErrorBackgroundEvent(e)); + return result; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 65ac7c983c7..d3004dd9227 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -48,10 +48,12 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY; + public class CommitRequestManager implements RequestManager { + // TODO: current in ConsumerConfig but inaccessible in the internal package. private static final String THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED = "internal.throw.on.fetch.stable.offset.unsupported"; - // TODO: We will need to refactor the subscriptionState private final SubscriptionState subscriptions; private final LogContext logContext; private final Logger log; @@ -96,17 +98,14 @@ public class CommitRequestManager implements RequestManager { @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { // poll only when the coordinator node is known. - if (!coordinatorRequestManager.coordinator().isPresent()) { - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList()); - } + if (!coordinatorRequestManager.coordinator().isPresent()) + return EMPTY; maybeAutoCommit(this.subscriptions.allConsumed()); - if (!pendingRequests.hasUnsentRequests()) { - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList()); - } + if (!pendingRequests.hasUnsentRequests()) + return EMPTY; - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, - Collections.unmodifiableList(pendingRequests.drain(currentTimeMs))); + return new NetworkClientDelegate.PollResult(pendingRequests.drain(currentTimeMs)); } public void maybeAutoCommit(final Map offsets) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java index 7a8ee105157..8959345bffd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java @@ -53,7 +53,7 @@ import java.util.Set; /** * {@link CompletedFetch} represents a {@link RecordBatch batch} of {@link Record records} that was returned from the * broker via a {@link FetchRequest}. It contains logic to maintain state between calls to - * {@link #fetchRecords(FetchConfig, int)}. + * {@link #fetchRecords(FetchConfig, Deserializers, int)}. */ public class CompletedFetch { @@ -135,7 +135,8 @@ public class CompletedFetch { /** * Draining a {@link CompletedFetch} will signal that the data has been consumed and the underlying resources * are closed. This is somewhat analogous to {@link Closeable#close() closing}, though no error will result if a - * caller invokes {@link #fetchRecords(FetchConfig, int)}; an empty {@link List list} will be returned instead. + * caller invokes {@link #fetchRecords(FetchConfig, Deserializers, int)}; an empty {@link List list} will be + * returned instead. */ void drain() { if (!isConsumed) { @@ -151,7 +152,7 @@ public class CompletedFetch { } } - private void maybeEnsureValid(FetchConfig fetchConfig, RecordBatch batch) { + private void maybeEnsureValid(FetchConfig fetchConfig, RecordBatch batch) { if (fetchConfig.checkCrcs && batch.magic() >= RecordBatch.MAGIC_VALUE_V2) { try { batch.ensureValid(); @@ -162,7 +163,7 @@ public class CompletedFetch { } } - private void maybeEnsureValid(FetchConfig fetchConfig, Record record) { + private void maybeEnsureValid(FetchConfig fetchConfig, Record record) { if (fetchConfig.checkCrcs) { try { record.ensureValid(); @@ -180,7 +181,7 @@ public class CompletedFetch { } } - private Record nextFetchedRecord(FetchConfig fetchConfig) { + private Record nextFetchedRecord(FetchConfig fetchConfig) { while (true) { if (records == null || !records.hasNext()) { maybeCloseRecordStream(); @@ -245,11 +246,14 @@ public class CompletedFetch { * {@link Deserializer deserialization} of the {@link Record record's} key and value are performed in * this step. * - * @param fetchConfig {@link FetchConfig Configuration} to use, including, but not limited to, {@link Deserializer}s + * @param fetchConfig {@link FetchConfig Configuration} to use + * @param deserializers {@link Deserializer}s to use to convert the raw bytes to the expected key and value types * @param maxRecords The number of records to return; the number returned may be {@code 0 <= maxRecords} * @return {@link ConsumerRecord Consumer records} */ - List> fetchRecords(FetchConfig fetchConfig, int maxRecords) { + List> fetchRecords(FetchConfig fetchConfig, + Deserializers deserializers, + int maxRecords) { // Error when fetching the next record before deserialization. if (corruptLastRecord) throw new KafkaException("Received exception when fetching the next record from " + partition @@ -276,7 +280,7 @@ public class CompletedFetch { Optional leaderEpoch = maybeLeaderEpoch(currentBatch.partitionLeaderEpoch()); TimestampType timestampType = currentBatch.timestampType(); - ConsumerRecord record = parseRecord(fetchConfig, partition, leaderEpoch, timestampType, lastRecord); + ConsumerRecord record = parseRecord(deserializers, partition, leaderEpoch, timestampType, lastRecord); records.add(record); recordsRead++; bytesRead += lastRecord.sizeInBytes(); @@ -302,7 +306,7 @@ public class CompletedFetch { /** * Parse the record entry, deserializing the key / value fields if necessary */ - ConsumerRecord parseRecord(FetchConfig fetchConfig, + ConsumerRecord parseRecord(Deserializers deserializers, TopicPartition partition, Optional leaderEpoch, TimestampType timestampType, @@ -312,16 +316,16 @@ public class CompletedFetch { long timestamp = record.timestamp(); Headers headers = new RecordHeaders(record.headers()); ByteBuffer keyBytes = record.key(); - K key = keyBytes == null ? null : fetchConfig.deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); + K key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); ByteBuffer valueBytes = record.value(); - V value = valueBytes == null ? null : fetchConfig.deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); + V value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, timestamp, timestampType, keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(), valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(), key, value, headers, leaderEpoch); } catch (RuntimeException e) { - log.error("Deserializers with error: {}", fetchConfig.deserializers); + log.error("Deserializers with error: {}", deserializers); throw new RecordDeserializationException(partition, record.offset(), "Error deserializing key/value for partition " + partition + " at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index b906a02fef9..bdcbfc39dfc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -955,10 +955,16 @@ public final class ConsumerCoordinator extends AbstractCoordinator { * @param timer Timer bounding how long this method can block * @return true iff the operation completed within the timeout */ - public boolean refreshCommittedOffsetsIfNeeded(Timer timer) { + public boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final Set initializingPartitions = subscriptions.initializingPartitions(); final Map offsets = fetchCommittedOffsets(initializingPartitions, timer); - return refreshCommittedOffsets(offsets, this.metadata, this.subscriptions); + + // "offsets" will be null if the offset fetch requests did not receive responses within the given timeout + if (offsets == null) + return false; + + refreshCommittedOffsets(offsets, this.metadata, this.subscriptions); + return true; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java new file mode 100644 index 00000000000..77a2952d1b2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -0,0 +1,268 @@ +/* + * 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.KafkaClient; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.internals.IdempotentCloser; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.utils.KafkaThread; +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; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.Future; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; +import static org.apache.kafka.common.utils.Utils.closeQuietly; + +/** + * Background thread runnable that consumes {@link ApplicationEvent} and produces {@link BackgroundEvent}. It + * uses an event loop to consume and produce events, and poll the network client to handle network IO. + */ +public class ConsumerNetworkThread extends KafkaThread implements Closeable { + + private static final long MAX_POLL_TIMEOUT_MS = 5000; + private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread"; + private final Time time; + private final Logger log; + private final Supplier applicationEventProcessorSupplier; + private final Supplier networkClientDelegateSupplier; + private final Supplier requestManagersSupplier; + private ApplicationEventProcessor applicationEventProcessor; + private NetworkClientDelegate networkClientDelegate; + private RequestManagers requestManagers; + private volatile boolean running; + private final IdempotentCloser closer = new IdempotentCloser(); + private volatile Duration closeTimeout = Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS); + + public ConsumerNetworkThread(LogContext logContext, + Time time, + Supplier applicationEventProcessorSupplier, + Supplier networkClientDelegateSupplier, + Supplier requestManagersSupplier) { + super(BACKGROUND_THREAD_NAME, true); + this.time = time; + this.log = logContext.logger(getClass()); + this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; + this.networkClientDelegateSupplier = networkClientDelegateSupplier; + this.requestManagersSupplier = requestManagersSupplier; + } + + @Override + public void run() { + closer.assertOpen("Consumer network thread is already closed"); + running = true; + + try { + log.debug("Consumer network thread started"); + + // Wait until we're securely in the background network thread to initialize these objects... + initializeResources(); + + while (running) { + try { + runOnce(); + } catch (final WakeupException e) { + log.debug("WakeupException caught, consumer network thread won't be interrupted"); + // swallow the wakeup exception to prevent killing the thread. + } + } + } catch (final Throwable t) { + log.error("The consumer network thread failed due to unexpected error", t); + throw new KafkaException(t); + } finally { + cleanup(); + } + } + + void initializeResources() { + applicationEventProcessor = applicationEventProcessorSupplier.get(); + networkClientDelegate = networkClientDelegateSupplier.get(); + requestManagers = requestManagersSupplier.get(); + } + + /** + * Poll and process the {@link ApplicationEvent application events}. It performs the following tasks: + * + *

    + *
  1. + * Drains and processes all the events from the application thread's application event queue via + * {@link ApplicationEventProcessor} + *
  2. + *
  3. + * Iterate through the {@link RequestManager} list and invoke {@link RequestManager#poll(long)} to get + * the {@link NetworkClientDelegate.UnsentRequest} list and the poll time for the network poll + *
  4. + *
  5. + * Stage each {@link AbstractRequest.Builder request} to be sent via + * {@link NetworkClientDelegate#addAll(List)} + *
  6. + *
  7. + * Poll the client via {@link KafkaClient#poll(long, long)} to send the requests, as well as + * retrieve any available responses + *
  8. + *
+ */ + void runOnce() { + // If there are errors processing any events, the error will be thrown immediately. This will have + // the effect of closing the background thread. + applicationEventProcessor.process(); + + final long currentTimeMs = time.milliseconds(); + final long pollWaitTimeMs = requestManagers.entries().stream() + .filter(Optional::isPresent) + .map(Optional::get) + .map(rm -> rm.poll(currentTimeMs)) + .map(networkClientDelegate::addAll) + .reduce(MAX_POLL_TIMEOUT_MS, Math::min); + networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs); + } + + /** + * Performs any network I/O that is needed at the time of close for the consumer: + * + *
    + *
  1. + * Iterate through the {@link RequestManager} list and invoke {@link RequestManager#pollOnClose()} + * to get the {@link NetworkClientDelegate.UnsentRequest} list and the poll time for the network poll + *
  2. + *
  3. + * Stage each {@link AbstractRequest.Builder request} to be sent via + * {@link NetworkClientDelegate#addAll(List)} + *
  4. + *
  5. + * {@link KafkaClient#poll(long, long) Poll the client} to send the requests, as well as + * retrieve any available responses + *
  6. + *
  7. + * Continuously {@link KafkaClient#poll(long, long) poll the client} as long as the + * {@link Timer#notExpired() timer hasn't expired} to retrieve the responses + *
  8. + *
+ */ + // Visible for testing + static void runAtClose(final Collection> requestManagers, + final NetworkClientDelegate networkClientDelegate, + final Timer timer) { + // These are the optional outgoing requests at the + List pollResults = requestManagers.stream() + .filter(Optional::isPresent) + .map(Optional::get) + .map(RequestManager::pollOnClose) + .collect(Collectors.toList()); + long pollWaitTimeMs = pollResults.stream() + .map(networkClientDelegate::addAll) + .reduce(MAX_POLL_TIMEOUT_MS, Math::min); + pollWaitTimeMs = Math.min(pollWaitTimeMs, timer.remainingMs()); + networkClientDelegate.poll(pollWaitTimeMs, timer.currentTimeMs()); + timer.update(); + + List> requestFutures = pollResults.stream() + .flatMap(fads -> fads.unsentRequests.stream()) + .map(NetworkClientDelegate.UnsentRequest::future) + .collect(Collectors.toList()); + + // Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until + // all requests have received a response. + while (timer.notExpired() && !requestFutures.stream().allMatch(Future::isDone)) { + networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs()); + timer.update(); + } + } + + public boolean isRunning() { + return running; + } + + public void wakeup() { + // The network client can be null if the initializeResources method has not yet been called. + if (networkClientDelegate != null) + networkClientDelegate.wakeup(); + } + + @Override + public void close() { + close(closeTimeout); + } + + public void close(final Duration timeout) { + Objects.requireNonNull(timeout, "Close timeout for consumer network thread must be non-null"); + + closer.close( + () -> closeInternal(timeout), + () -> log.warn("The consumer network thread was already closed") + ); + } + + /** + * Starts the closing process. + * + *

+ * + * This method is called from the application thread, but our resources are owned by the network thread. As such, + * we don't actually close any of those resources here, immediately, on the application thread. Instead, we just + * update our internal state on the application thread. When the network thread next + * {@link #run() executes its loop}, it will notice that state, cease processing any further events, and begin + * {@link #cleanup() closing its resources}. + * + *

+ * + * This method will wait (i.e. block the application thread) for up to the duration of the given timeout to give + * the network thread the time to close down cleanly. + * + * @param timeout Upper bound of time to wait for the network thread to close its resources + */ + private void closeInternal(final Duration timeout) { + long timeoutMs = timeout.toMillis(); + log.trace("Signaling the consumer network thread to close in {}ms", timeoutMs); + running = false; + closeTimeout = timeout; + wakeup(); + + if (timeoutMs > 0) { + try { + join(timeoutMs); + } catch (InterruptedException e) { + log.error("Interrupted while waiting for consumer network thread to complete", e); + } + } + } + + void cleanup() { + log.trace("Closing the consumer network thread"); + Timer timer = time.timer(closeTimeout); + runAtClose(requestManagers.entries(), networkClientDelegate, timer); + closeQuietly(requestManagers, "request managers"); + closeQuietly(networkClientDelegate, "network client delegate"); + closeQuietly(applicationEventProcessor, "application event processor"); + log.debug("Closed the consumer network thread"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 68d2458e28d..92b098213b0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -56,6 +56,7 @@ import java.util.concurrent.TimeUnit; public final class ConsumerUtils { + public static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000; public static final String CONSUMER_JMX_PREFIX = "kafka.consumer"; public static final String CONSUMER_METRIC_GROUP_PREFIX = "consumer"; @@ -142,12 +143,6 @@ public final class ConsumerUtils { return new FetchMetricsManager(metrics, metricsRegistry); } - public static FetchConfig createFetchConfig(ConsumerConfig config, - Deserializers deserializers) { - IsolationLevel isolationLevel = configuredIsolationLevel(config); - return new FetchConfig<>(config, deserializers, isolationLevel); - } - @SuppressWarnings("unchecked") public static List> configuredConsumerInterceptors(ConsumerConfig config) { return (List>) ClientUtils.configuredInterceptors(config, ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, ConsumerInterceptor.class); @@ -168,14 +163,10 @@ public final class ConsumerUtils { * committed offsets' metadata. * @param subscriptions Subscription state to update, setting partitions' offsets to the * committed offsets. - * @return False if null offsetsAndMetadata is provided, indicating that the - * refresh operation could not be performed. True in any other case. */ - public static boolean refreshCommittedOffsets(final Map offsetsAndMetadata, - final ConsumerMetadata metadata, - final SubscriptionState subscriptions) { - if (offsetsAndMetadata == null) return false; - + public static void refreshCommittedOffsets(final Map offsetsAndMetadata, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions) { for (final Map.Entry entry : offsetsAndMetadata.entrySet()) { final TopicPartition tp = entry.getKey(); final OffsetAndMetadata offsetAndMetadata = entry.getValue(); @@ -200,7 +191,6 @@ public final class ConsumerUtils { } } } - return true; } public static T getResult(CompletableFuture future, Timer timer) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index c67bbeaa34e..3efe45499f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; @@ -28,10 +31,11 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; -import java.util.Collections; import java.util.Objects; import java.util.Optional; +import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY; + /** * This is responsible for timing to send the next {@link FindCoordinatorRequest} based on the following criteria: *

@@ -49,7 +53,7 @@ public class CoordinatorRequestManager implements RequestManager { private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000; private final Time time; private final Logger log; - private final ErrorEventHandler nonRetriableErrorHandler; + private final BackgroundEventHandler backgroundEventHandler; private final String groupId; private final RequestState coordinatorRequestState; @@ -62,13 +66,13 @@ public class CoordinatorRequestManager implements RequestManager { final LogContext logContext, final long retryBackoffMs, final long retryBackoffMaxMs, - final ErrorEventHandler errorHandler, + final BackgroundEventHandler errorHandler, final String groupId ) { Objects.requireNonNull(groupId); this.time = time; this.log = logContext.logger(this.getClass()); - this.nonRetriableErrorHandler = errorHandler; + this.backgroundEventHandler = errorHandler; this.groupId = groupId; this.coordinatorRequestState = new RequestState( logContext, @@ -90,18 +94,15 @@ public class CoordinatorRequestManager implements RequestManager { */ @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { - if (this.coordinator != null) { - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList()); - } + if (this.coordinator != null) + return EMPTY; if (coordinatorRequestState.canSendRequest(currentTimeMs)) { NetworkClientDelegate.UnsentRequest request = makeFindCoordinatorRequest(currentTimeMs); - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.singletonList(request)); + return new NetworkClientDelegate.PollResult(request); } - return new NetworkClientDelegate.PollResult( - coordinatorRequestState.remainingBackoffMs(currentTimeMs), - Collections.emptyList()); + return new NetworkClientDelegate.PollResult(coordinatorRequestState.remainingBackoffMs(currentTimeMs)); } private NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) { @@ -175,12 +176,13 @@ public class CoordinatorRequestManager implements RequestManager { if (exception == Errors.GROUP_AUTHORIZATION_FAILED.exception()) { log.debug("FindCoordinator request failed due to authorization error {}", exception.getMessage()); - nonRetriableErrorHandler.handle(GroupAuthorizationException.forGroupId(this.groupId)); + KafkaException groupAuthorizationException = GroupAuthorizationException.forGroupId(this.groupId); + backgroundEventHandler.add(new ErrorBackgroundEvent(groupAuthorizationException)); return; } log.warn("FindCoordinator request failed due to fatal exception", exception); - nonRetriableErrorHandler.handle(exception); + backgroundEventHandler.add(new ErrorBackgroundEvent(exception)); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java deleted file mode 100644 index 74322e6fec0..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java +++ /dev/null @@ -1,293 +0,0 @@ -/* - * 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.ClientUtils; -import org.apache.kafka.clients.GroupRebalanceConfig; -import org.apache.kafka.clients.NetworkClient; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; - -import java.util.LinkedList; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; - -import static java.util.Objects.requireNonNull; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel; - -/** - * Background thread runnable that consumes {@code ApplicationEvent} and - * produces {@code BackgroundEvent}. It uses an event loop to consume and - * produce events, and poll the network client to handle network IO. - *

- * It holds a reference to the {@link SubscriptionState}, which is - * initialized by the polling thread. - *

- * For processing application events that have been submitted to the - * {@link #applicationEventQueue}, this relies on an {@link ApplicationEventProcessor}. Processing includes generating requests and - * handling responses with the appropriate {@link RequestManager}. The network operations for - * actually sending the requests is delegated to the {@link NetworkClientDelegate} - * - */ -public class DefaultBackgroundThread extends KafkaThread { - private static final long MAX_POLL_TIMEOUT_MS = 5000; - private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread"; - private final Time time; - private final Logger log; - private final BlockingQueue applicationEventQueue; - private final BlockingQueue backgroundEventQueue; - private final ConsumerMetadata metadata; - private final ConsumerConfig config; - // empty if groupId is null - private final ApplicationEventProcessor applicationEventProcessor; - private final NetworkClientDelegate networkClientDelegate; - private final ErrorEventHandler errorEventHandler; - private final GroupState groupState; - private boolean running; - - private final RequestManagers requestManagers; - - // Visible for testing - @SuppressWarnings("ParameterNumber") - DefaultBackgroundThread(final Time time, - final ConsumerConfig config, - final LogContext logContext, - final BlockingQueue applicationEventQueue, - final BlockingQueue backgroundEventQueue, - final ErrorEventHandler errorEventHandler, - final ApplicationEventProcessor processor, - final ConsumerMetadata metadata, - final NetworkClientDelegate networkClient, - final GroupState groupState, - final CoordinatorRequestManager coordinatorManager, - final CommitRequestManager commitRequestManager, - final OffsetsRequestManager offsetsRequestManager, - final TopicMetadataRequestManager topicMetadataRequestManager, - final HeartbeatRequestManager heartbeatRequestManager) { - super(BACKGROUND_THREAD_NAME, true); - this.time = time; - this.running = true; - this.log = logContext.logger(getClass()); - this.applicationEventQueue = applicationEventQueue; - this.backgroundEventQueue = backgroundEventQueue; - this.applicationEventProcessor = processor; - this.config = config; - this.metadata = metadata; - this.networkClientDelegate = networkClient; - this.errorEventHandler = errorEventHandler; - this.groupState = groupState; - this.requestManagers = new RequestManagers( - offsetsRequestManager, - topicMetadataRequestManager, - Optional.ofNullable(coordinatorManager), - Optional.ofNullable(commitRequestManager), - Optional.ofNullable(heartbeatRequestManager)); - } - - public DefaultBackgroundThread(final Time time, - final ConsumerConfig config, - final GroupRebalanceConfig rebalanceConfig, - final LogContext logContext, - final BlockingQueue applicationEventQueue, - final BlockingQueue backgroundEventQueue, - final ConsumerMetadata metadata, - final SubscriptionState subscriptionState, - final ApiVersions apiVersions, - final Metrics metrics, - final Sensor fetcherThrottleTimeSensor) { - super(BACKGROUND_THREAD_NAME, true); - requireNonNull(config); - requireNonNull(rebalanceConfig); - requireNonNull(logContext); - requireNonNull(applicationEventQueue); - requireNonNull(backgroundEventQueue); - requireNonNull(metadata); - requireNonNull(subscriptionState); - try { - this.time = time; - this.log = logContext.logger(getClass()); - this.applicationEventQueue = applicationEventQueue; - this.backgroundEventQueue = backgroundEventQueue; - this.config = config; - this.metadata = metadata; - final NetworkClient networkClient = ClientUtils.createNetworkClient(config, - metrics, - CONSUMER_METRIC_GROUP_PREFIX, - logContext, - apiVersions, - time, - CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, - metadata, - fetcherThrottleTimeSensor); - this.networkClientDelegate = new NetworkClientDelegate( - this.time, - this.config, - logContext, - networkClient); - this.running = true; - this.errorEventHandler = new ErrorEventHandler(this.backgroundEventQueue); - this.groupState = new GroupState(rebalanceConfig); - long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); - final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); - - OffsetsRequestManager offsetsRequestManager = - new OffsetsRequestManager( - subscriptionState, - metadata, - configuredIsolationLevel(config), - time, - retryBackoffMs, - requestTimeoutMs, - apiVersions, - networkClientDelegate, - logContext); - CoordinatorRequestManager coordinatorRequestManager = null; - CommitRequestManager commitRequestManager = null; - TopicMetadataRequestManager topicMetadataRequestManger = new TopicMetadataRequestManager( - logContext, - config); - HeartbeatRequestManager heartbeatRequestManager = null; - - // TODO: consolidate groupState and memberState - if (groupState.groupId != null) { - coordinatorRequestManager = new CoordinatorRequestManager( - this.time, - logContext, - retryBackoffMs, - retryBackoffMaxMs, - this.errorEventHandler, - groupState.groupId); - commitRequestManager = new CommitRequestManager( - this.time, - logContext, - subscriptionState, - config, - coordinatorRequestManager, - groupState); - MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext); - heartbeatRequestManager = new HeartbeatRequestManager( - this.time, - logContext, - config, - coordinatorRequestManager, - subscriptionState, - membershipManager, - errorEventHandler); - } - - this.requestManagers = new RequestManagers( - offsetsRequestManager, - topicMetadataRequestManger, - Optional.ofNullable(coordinatorRequestManager), - Optional.ofNullable(commitRequestManager), - Optional.ofNullable(heartbeatRequestManager)); - this.applicationEventProcessor = new ApplicationEventProcessor( - backgroundEventQueue, - requestManagers, - metadata); - } catch (final Exception e) { - close(); - throw new KafkaException("Failed to construct background processor", e.getCause()); - } - } - - @Override - public void run() { - try { - log.debug("Background thread started"); - while (running) { - try { - runOnce(); - } catch (final WakeupException e) { - log.debug("WakeupException caught, background thread won't be interrupted"); - // swallow the wakeup exception to prevent killing the background thread. - } - } - } catch (final Throwable t) { - log.error("The background thread failed due to unexpected error", t); - throw new KafkaException(t); - } finally { - close(); - log.debug("{} closed", getClass()); - } - } - - /** - * Poll and process an {@link ApplicationEvent}. It performs the following tasks: - * 1. Drains and try to process all the requests in the queue. - * 2. Iterate through the registry, poll, and get the next poll time for the network poll - * 3. Poll the networkClient to send and retrieve the response. - */ - void runOnce() { - if (!applicationEventQueue.isEmpty()) { - LinkedList res = new LinkedList<>(); - this.applicationEventQueue.drainTo(res); - - for (ApplicationEvent event : res) { - log.debug("Consuming application event: {}", event); - Objects.requireNonNull(event); - applicationEventProcessor.process(event); - } - } - - final long currentTimeMs = time.milliseconds(); - final long pollWaitTimeMs = requestManagers.entries().stream() - .filter(Optional::isPresent) - .map(m -> m.get().poll(currentTimeMs)) - .map(this::handlePollResult) - .reduce(MAX_POLL_TIMEOUT_MS, Math::min); - networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs); - } - - long handlePollResult(NetworkClientDelegate.PollResult res) { - if (!res.unsentRequests.isEmpty()) { - networkClientDelegate.addAll(res.unsentRequests); - } - return res.timeUntilNextPollMs; - } - - public boolean isRunning() { - return this.running; - } - - public final void wakeup() { - networkClientDelegate.wakeup(); - } - - public final void close() { - this.running = false; - this.wakeup(); - Utils.closeQuietly(networkClientDelegate, "network client utils"); - Utils.closeQuietly(metadata, "consumer metadata client"); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandler.java deleted file mode 100644 index 4d620a4ffac..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandler.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * 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.ClientUtils; -import org.apache.kafka.clients.GroupRebalanceConfig; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.EventHandler; -import org.apache.kafka.common.internals.ClusterResourceListeners; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; - -import java.net.InetSocketAddress; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -/** - * An {@code EventHandler} that uses a single background thread to consume {@code ApplicationEvent} and produce - * {@code BackgroundEvent} from the {@link DefaultBackgroundThread}. - */ -public class DefaultEventHandler implements EventHandler { - - private final BlockingQueue applicationEventQueue; - private final BlockingQueue backgroundEventQueue; - private final DefaultBackgroundThread backgroundThread; - - public DefaultEventHandler(final ConsumerConfig config, - final GroupRebalanceConfig groupRebalanceConfig, - final LogContext logContext, - final SubscriptionState subscriptionState, - final ApiVersions apiVersions, - final Metrics metrics, - final ClusterResourceListeners clusterResourceListeners, - final Sensor fetcherThrottleTimeSensor) { - this(Time.SYSTEM, - config, - groupRebalanceConfig, - logContext, - new LinkedBlockingQueue<>(), - new LinkedBlockingQueue<>(), - subscriptionState, - apiVersions, - metrics, - clusterResourceListeners, - fetcherThrottleTimeSensor); - } - - public DefaultEventHandler(final Time time, - final ConsumerConfig config, - final GroupRebalanceConfig groupRebalanceConfig, - final LogContext logContext, - final BlockingQueue applicationEventQueue, - final BlockingQueue backgroundEventQueue, - final SubscriptionState subscriptionState, - final ApiVersions apiVersions, - final Metrics metrics, - final ClusterResourceListeners clusterResourceListeners, - final Sensor fetcherThrottleTimeSensor) { - this.applicationEventQueue = applicationEventQueue; - this.backgroundEventQueue = backgroundEventQueue; - - // Bootstrap a metadata object with the bootstrap server IP address, which will be used once for the - // subsequent metadata refresh once the background thread has started up. - final ConsumerMetadata metadata = new ConsumerMetadata(config, - subscriptionState, - logContext, - clusterResourceListeners); - final List addresses = ClientUtils.parseAndValidateAddresses(config); - metadata.bootstrap(addresses); - - this.backgroundThread = new DefaultBackgroundThread( - time, - config, - groupRebalanceConfig, - logContext, - this.applicationEventQueue, - this.backgroundEventQueue, - metadata, - subscriptionState, - apiVersions, - metrics, - fetcherThrottleTimeSensor); - backgroundThread.start(); - } - - // VisibleForTesting - DefaultEventHandler(final DefaultBackgroundThread backgroundThread, - final BlockingQueue applicationEventQueue, - final BlockingQueue backgroundEventQueue) { - this.backgroundThread = backgroundThread; - this.applicationEventQueue = applicationEventQueue; - this.backgroundEventQueue = backgroundEventQueue; - backgroundThread.start(); - } - - @Override - public Optional poll() { - return Optional.ofNullable(backgroundEventQueue.poll()); - } - - @Override - public boolean isEmpty() { - return backgroundEventQueue.isEmpty(); - } - - @Override - public boolean add(final ApplicationEvent event) { - backgroundThread.wakeup(); - return applicationEventQueue.add(event); - } - - @Override - public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { - Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); - Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); - add(event); - return event.get(timer); - } - - public void close() { - try { - backgroundThread.close(); - } catch (final Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Deserializers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Deserializers.java index 80ba5cb7d9e..5de2a888775 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Deserializers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Deserializers.java @@ -75,4 +75,12 @@ public class Deserializers implements AutoCloseable { throw new KafkaException("Failed to close deserializers", exception); } } + + @Override + public String toString() { + return "Deserializers{" + + "keyDeserializer=" + keyDeserializer + + ", valueDeserializer=" + valueDeserializer + + '}'; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java index d82b37ad99a..de7f88ab725 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java @@ -17,15 +17,21 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Timer; import org.slf4j.Logger; -import java.io.Closeable; import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; /** @@ -35,12 +41,15 @@ import java.util.function.Predicate; * *

* - * Note: this class is not thread-safe and is intended to only be used from a single thread. + * Note: this class is thread-safe with the intention that {@link CompletedFetch the data} will be + * "produced" by a background thread and consumed by the application thread. */ -public class FetchBuffer implements Closeable { +public class FetchBuffer implements AutoCloseable { private final Logger log; private final ConcurrentLinkedQueue completedFetches; + private final Lock lock; + private final Condition notEmptyCondition; private final IdempotentCloser idempotentCloser = new IdempotentCloser(); private CompletedFetch nextInLineFetch; @@ -48,6 +57,8 @@ public class FetchBuffer implements Closeable { public FetchBuffer(final LogContext logContext) { this.log = logContext.logger(FetchBuffer.class); this.completedFetches = new ConcurrentLinkedQueue<>(); + this.lock = new ReentrantLock(); + this.notEmptyCondition = lock.newCondition(); } /** @@ -56,7 +67,12 @@ public class FetchBuffer implements Closeable { * @return {@code true} if the buffer is empty, {@code false} otherwise */ boolean isEmpty() { - return completedFetches.isEmpty(); + try { + lock.lock(); + return completedFetches.isEmpty(); + } finally { + lock.unlock(); + } } /** @@ -66,31 +82,107 @@ public class FetchBuffer implements Closeable { * @return {@code true} if there are completed fetches that match the {@link Predicate}, {@code false} otherwise */ boolean hasCompletedFetches(Predicate predicate) { - return completedFetches.stream().anyMatch(predicate); + try { + lock.lock(); + return completedFetches.stream().anyMatch(predicate); + } finally { + lock.unlock(); + } } void add(CompletedFetch completedFetch) { - completedFetches.add(completedFetch); + try { + lock.lock(); + completedFetches.add(completedFetch); + notEmptyCondition.signalAll(); + } finally { + lock.unlock(); + } } void addAll(Collection completedFetches) { - this.completedFetches.addAll(completedFetches); + if (completedFetches == null || completedFetches.isEmpty()) + return; + + try { + lock.lock(); + this.completedFetches.addAll(completedFetches); + notEmptyCondition.signalAll(); + } finally { + lock.unlock(); + } } CompletedFetch nextInLineFetch() { - return nextInLineFetch; + try { + lock.lock(); + return nextInLineFetch; + } finally { + lock.unlock(); + } } - void setNextInLineFetch(CompletedFetch completedFetch) { - this.nextInLineFetch = completedFetch; + void setNextInLineFetch(CompletedFetch nextInLineFetch) { + try { + lock.lock(); + this.nextInLineFetch = nextInLineFetch; + } finally { + lock.unlock(); + } } CompletedFetch peek() { - return completedFetches.peek(); + try { + lock.lock(); + return completedFetches.peek(); + } finally { + lock.unlock(); + } } CompletedFetch poll() { - return completedFetches.poll(); + try { + lock.lock(); + return completedFetches.poll(); + } finally { + lock.unlock(); + } + } + + /** + * Allows the caller to await presence of data in the buffer. The method will block, returning only + * under one of the following conditions: + * + *

    + *
  1. The buffer was already non-empty on entry
  2. + *
  3. The buffer was populated during the wait
  4. + *
  5. The remaining time on the {@link Timer timer} elapsed
  6. + *
  7. The thread was interrupted
  8. + *
+ * + * @param timer Timer that provides time to wait + */ + void awaitNotEmpty(Timer timer) { + try { + lock.lock(); + + while (isEmpty()) { + // Update the timer before we head into the loop in case it took a while to get the lock. + timer.update(); + + if (timer.isExpired()) + break; + + if (!notEmptyCondition.await(timer.remainingMs(), TimeUnit.MILLISECONDS)) { + break; + } + } + } catch (InterruptedException e) { + throw new InterruptException("Timeout waiting for results from fetching records", e); + } finally { + lock.unlock(); + timer.update(); + } } /** @@ -100,12 +192,22 @@ public class FetchBuffer implements Closeable { * @param partitions {@link Set} of {@link TopicPartition}s for which any buffered data should be kept */ void retainAll(final Set partitions) { - completedFetches.removeIf(cf -> maybeDrain(partitions, cf)); + try { + lock.lock(); + + completedFetches.removeIf(cf -> maybeDrain(partitions, cf)); - if (maybeDrain(partitions, nextInLineFetch)) - nextInLineFetch = null; + if (maybeDrain(partitions, nextInLineFetch)) + nextInLineFetch = null; + } finally { + lock.unlock(); + } } + /** + * Drains (i.e. removes) the contents of the given {@link CompletedFetch} as its data should not + * be returned to the user. + */ private boolean maybeDrain(final Set partitions, final CompletedFetch completedFetch) { if (completedFetch != null && !partitions.contains(completedFetch.partition)) { log.debug("Removing {} from buffered fetch data as it is not in the set of partitions to retain ({})", completedFetch.partition, partitions); @@ -122,28 +224,33 @@ public class FetchBuffer implements Closeable { * @return {@link TopicPartition Partition} set */ Set bufferedPartitions() { - final Set partitions = new HashSet<>(); + try { + lock.lock(); - if (nextInLineFetch != null && !nextInLineFetch.isConsumed()) { - partitions.add(nextInLineFetch.partition); - } + final Set partitions = new HashSet<>(); + + if (nextInLineFetch != null && !nextInLineFetch.isConsumed()) { + partitions.add(nextInLineFetch.partition); + } - completedFetches.forEach(cf -> partitions.add(cf.partition)); - return partitions; + completedFetches.forEach(cf -> partitions.add(cf.partition)); + return partitions; + } finally { + lock.unlock(); + } } @Override public void close() { - idempotentCloser.close(() -> { - log.debug("Closing the fetch buffer"); + try { + lock.lock(); - if (nextInLineFetch != null) { - nextInLineFetch.drain(); - nextInLineFetch = null; - } - - completedFetches.forEach(CompletedFetch::drain); - completedFetches.clear(); - }, () -> log.warn("The fetch buffer was previously closed")); + idempotentCloser.close( + () -> retainAll(Collections.emptySet()), + () -> log.warn("The fetch buffer was already closed") + ); + } finally { + lock.unlock(); + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java index 0e1f2f18a3f..e98441321d3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java @@ -54,20 +54,23 @@ public class FetchCollector { private final Logger log; private final ConsumerMetadata metadata; private final SubscriptionState subscriptions; - private final FetchConfig fetchConfig; + private final FetchConfig fetchConfig; + private final Deserializers deserializers; private final FetchMetricsManager metricsManager; private final Time time; public FetchCollector(final LogContext logContext, final ConsumerMetadata metadata, final SubscriptionState subscriptions, - final FetchConfig fetchConfig, + final FetchConfig fetchConfig, + final Deserializers deserializers, final FetchMetricsManager metricsManager, final Time time) { this.log = logContext.logger(FetchCollector.class); this.metadata = metadata; this.subscriptions = subscriptions; this.fetchConfig = fetchConfig; + this.deserializers = deserializers; this.metricsManager = metricsManager; this.time = time; } @@ -162,7 +165,9 @@ public class FetchCollector { throw new IllegalStateException("Missing position for fetchable partition " + tp); if (nextInLineFetch.nextFetchOffset() == position.offset) { - List> partRecords = nextInLineFetch.fetchRecords(fetchConfig, fetchConfig.maxPollRecords); + List> partRecords = nextInLineFetch.fetchRecords(fetchConfig, + deserializers, + fetchConfig.maxPollRecords); log.trace("Returning {} fetched records at offset {} for assigned partition {}", partRecords.size(), position, tp); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchConfig.java index 66670b54472..24de49fee1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchConfig.java @@ -19,57 +19,30 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.IsolationLevel; -import org.apache.kafka.common.serialization.Deserializer; -import java.util.Objects; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel; /** * {@link FetchConfig} represents the static configuration for fetching records from Kafka. It is simply a way * to bundle the immutable settings that were presented at the time the {@link Consumer} was created for later use by * classes like {@link Fetcher}, {@link CompletedFetch}, etc. - * - *

- * - * In most cases, the values stored and returned by {@link FetchConfig} will be those stored in the following - * {@link ConsumerConfig consumer configuration} settings: - * - *

    - *
  • {@link #minBytes}: {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG}
  • - *
  • {@link #maxBytes}: {@link ConsumerConfig#FETCH_MAX_BYTES_CONFIG}
  • - *
  • {@link #maxWaitMs}: {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}
  • - *
  • {@link #fetchSize}: {@link ConsumerConfig#MAX_PARTITION_FETCH_BYTES_CONFIG}
  • - *
  • {@link #maxPollRecords}: {@link ConsumerConfig#MAX_POLL_RECORDS_CONFIG}
  • - *
  • {@link #checkCrcs}: {@link ConsumerConfig#CHECK_CRCS_CONFIG}
  • - *
  • {@link #clientRackId}: {@link ConsumerConfig#CLIENT_RACK_CONFIG}
  • - *
  • {@link #deserializers}: - * {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG}/{@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG} - *
  • - *
  • {@link #isolationLevel}: {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG}
  • - *
- * - * However, there are places in the code where additional logic is used to determine these fetch-related configuration - * values. In those cases, the values are calculated outside of this class and simply passed in when constructed. - * - *

- * - * Note: the {@link Deserializer deserializers} used for the key and value are not closed by this class. They should be - * closed by the creator of the {@link FetchConfig}. - * - * @param Type used to {@link Deserializer deserialize} the message/record key - * @param Type used to {@link Deserializer deserialize} the message/record value */ -public class FetchConfig { +public class FetchConfig { - final int minBytes; - final int maxBytes; - final int maxWaitMs; - final int fetchSize; - final int maxPollRecords; - final boolean checkCrcs; - final String clientRackId; - final Deserializers deserializers; - final IsolationLevel isolationLevel; + public final int minBytes; + public final int maxBytes; + public final int maxWaitMs; + public final int fetchSize; + public final int maxPollRecords; + public final boolean checkCrcs; + public final String clientRackId; + public final IsolationLevel isolationLevel; + /** + * Constructs a new {@link FetchConfig} using explicitly provided values. This is provided here for tests that + * want to exercise different scenarios can construct specific configuration values rather than going through + * the hassle of constructing a {@link ConsumerConfig}. + */ public FetchConfig(int minBytes, int maxBytes, int maxWaitMs, @@ -77,7 +50,6 @@ public class FetchConfig { int maxPollRecords, boolean checkCrcs, String clientRackId, - Deserializers deserializers, IsolationLevel isolationLevel) { this.minBytes = minBytes; this.maxBytes = maxBytes; @@ -86,13 +58,27 @@ public class FetchConfig { this.maxPollRecords = maxPollRecords; this.checkCrcs = checkCrcs; this.clientRackId = clientRackId; - this.deserializers = Objects.requireNonNull(deserializers, "Message deserializers provided to FetchConfig should not be null"); this.isolationLevel = isolationLevel; } - public FetchConfig(ConsumerConfig config, - Deserializers deserializers, - IsolationLevel isolationLevel) { + /** + * Constructs a new {@link FetchConfig} using values from the given {@link ConsumerConfig consumer configuration} + * settings: + * + *

    + *
  • {@link #minBytes}: {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG}
  • + *
  • {@link #maxBytes}: {@link ConsumerConfig#FETCH_MAX_BYTES_CONFIG}
  • + *
  • {@link #maxWaitMs}: {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}
  • + *
  • {@link #fetchSize}: {@link ConsumerConfig#MAX_PARTITION_FETCH_BYTES_CONFIG}
  • + *
  • {@link #maxPollRecords}: {@link ConsumerConfig#MAX_POLL_RECORDS_CONFIG}
  • + *
  • {@link #checkCrcs}: {@link ConsumerConfig#CHECK_CRCS_CONFIG}
  • + *
  • {@link #clientRackId}: {@link ConsumerConfig#CLIENT_RACK_CONFIG}
  • + *
  • {@link #isolationLevel}: {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG}
  • + *
+ * + * @param config Consumer configuration + */ + public FetchConfig(ConsumerConfig config) { this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG); this.maxBytes = config.getInt(ConsumerConfig.FETCH_MAX_BYTES_CONFIG); this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG); @@ -100,8 +86,7 @@ public class FetchConfig { this.maxPollRecords = config.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG); this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG); this.clientRackId = config.getString(ConsumerConfig.CLIENT_RACK_CONFIG); - this.deserializers = Objects.requireNonNull(deserializers, "Message deserializers provided to FetchConfig should not be null"); - this.isolationLevel = isolationLevel; + this.isolationLevel = configuredIsolationLevel(config); } @Override @@ -114,7 +99,6 @@ public class FetchConfig { ", maxPollRecords=" + maxPollRecords + ", checkCrcs=" + checkCrcs + ", clientRackId='" + clientRackId + '\'' + - ", deserializers=" + deserializers + ", isolationLevel=" + isolationLevel + '}'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java new file mode 100644 index 00000000000..f98d7b3d9fa --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java @@ -0,0 +1,117 @@ +/* + * 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.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.FetchSessionHandler; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +/** + * {@code FetchRequestManager} is responsible for generating {@link FetchRequest} that represent the + * {@link SubscriptionState#fetchablePartitions(Predicate)} based on the user's topic subscription/partition + * assignment. + */ +public class FetchRequestManager extends AbstractFetch implements RequestManager { + + private final NetworkClientDelegate networkClientDelegate; + + FetchRequestManager(final LogContext logContext, + final Time time, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final FetchConfig fetchConfig, + final FetchBuffer fetchBuffer, + final FetchMetricsManager metricsManager, + final NetworkClientDelegate networkClientDelegate) { + super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time); + this.networkClientDelegate = networkClientDelegate; + } + + @Override + protected boolean isUnavailable(Node node) { + return networkClientDelegate.isUnavailable(node); + } + + @Override + protected void maybeThrowAuthFailure(Node node) { + networkClientDelegate.maybeThrowAuthFailure(node); + } + + /** + * {@inheritDoc} + */ + @Override + public PollResult poll(long currentTimeMs) { + return pollInternal( + prepareFetchRequests(), + this::handleFetchSuccess, + this::handleFetchFailure + ); + } + + /** + * {@inheritDoc} + */ + @Override + public PollResult pollOnClose() { + return pollInternal( + prepareCloseFetchSessionRequests(), + this::handleCloseFetchSessionSuccess, + this::handleCloseFetchSessionFailure + ); + } + + /** + * Creates the {@link PollResult poll result} that contains a list of zero or more + * {@link FetchRequest.Builder fetch requests}. + * + * @param fetchRequests {@link Map} of {@link Node nodes} to their {@link FetchSessionHandler.FetchRequestData} + * @param successHandler {@link ResponseHandler Handler for successful responses} + * @param errorHandler {@link ResponseHandler Handler for failure responses} + * @return {@link PollResult} + */ + private PollResult pollInternal(Map fetchRequests, + ResponseHandler successHandler, + ResponseHandler errorHandler) { + List requests = fetchRequests.entrySet().stream().map(entry -> { + final Node fetchTarget = entry.getKey(); + final FetchSessionHandler.FetchRequestData data = entry.getValue(); + final FetchRequest.Builder request = createFetchRequest(fetchTarget, data); + final BiConsumer responseHandler = (clientResponse, error) -> { + if (error != null) + errorHandler.handle(fetchTarget, data, error); + else + successHandler.handle(fetchTarget, data, clientResponse); + }; + + return new UnsentRequest(request, fetchTarget, responseHandler); + }).collect(Collectors.toList()); + + return new PollResult(requests); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index dec02c6b90f..05df99373e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -16,16 +16,23 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.FetchSessionHandler; +import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.FetchRequest; 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; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Map; /** @@ -47,66 +54,147 @@ import java.util.Map; * on a different thread. * */ -public class Fetcher extends AbstractFetch { +public class Fetcher extends AbstractFetch { + private final Logger log; + private final ConsumerNetworkClient client; private final FetchCollector fetchCollector; public Fetcher(LogContext logContext, ConsumerNetworkClient client, ConsumerMetadata metadata, SubscriptionState subscriptions, - FetchConfig fetchConfig, + FetchConfig fetchConfig, + Deserializers deserializers, FetchMetricsManager metricsManager, Time time) { - super(logContext, client, metadata, subscriptions, fetchConfig, metricsManager, time); + super(logContext, metadata, subscriptions, fetchConfig, new FetchBuffer(logContext), metricsManager, time); + this.log = logContext.logger(Fetcher.class); + this.client = client; this.fetchCollector = new FetchCollector<>(logContext, metadata, subscriptions, fetchConfig, + deserializers, metricsManager, time); } + @Override + protected boolean isUnavailable(Node node) { + return client.isUnavailable(node); + } + + @Override + protected void maybeThrowAuthFailure(Node node) { + client.maybeThrowAuthFailure(node); + } + public void clearBufferedDataForUnassignedPartitions(Collection assignedPartitions) { fetchBuffer.retainAll(new HashSet<>(assignedPartitions)); } /** - * Set-up a fetch request for any node that we have assigned partitions for which doesn't already have + * Set up a fetch request for any node that we have assigned partitions for which doesn't already have * an in-flight fetch or pending fetch data. * @return number of fetches sent */ public synchronized int sendFetches() { - Map fetchRequestMap = prepareFetchRequests(); + final Map fetchRequests = prepareFetchRequests(); + sendFetchesInternal( + fetchRequests, + (fetchTarget, data, clientResponse) -> { + synchronized (Fetcher.this) { + handleFetchSuccess(fetchTarget, data, clientResponse); + } + }, + (fetchTarget, data, error) -> { + synchronized (Fetcher.this) { + handleFetchFailure(fetchTarget, data, error); + } + }); + return fetchRequests.size(); + } - for (Map.Entry entry : fetchRequestMap.entrySet()) { + protected void maybeCloseFetchSessions(final Timer timer) { + final List> requestFutures = sendFetchesInternal( + prepareCloseFetchSessionRequests(), + this::handleCloseFetchSessionSuccess, + this::handleCloseFetchSessionFailure + ); + + // Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until + // all requests have received a response. + while (timer.notExpired() && !requestFutures.stream().allMatch(RequestFuture::isDone)) { + client.poll(timer, null, true); + timer.update(); + } + + if (!requestFutures.stream().allMatch(RequestFuture::isDone)) { + // we ran out of time before completing all futures. It is ok since we don't want to block the shutdown + // here. + log.debug("All requests couldn't be sent in the specific timeout period {}ms. " + + "This may result in unnecessary fetch sessions at the broker. Consider increasing the timeout passed for " + + "KafkaConsumer.close(Duration timeout)", timer.timeoutMs()); + } + } + + public Fetch collectFetch() { + return fetchCollector.collectFetch(fetchBuffer); + } + + /** + * This method is called by {@link #close(Timer)} which is guarded by the {@link IdempotentCloser}) such as to only + * be executed once the first time that any of the {@link #close()} methods are called. Subclasses can override + * this method without the need for extra synchronization at the instance level. + * + * @param timer Timer to enforce time limit + */ + // Visible for testing + protected void closeInternal(Timer timer) { + // we do not need to re-enable wake-ups since we are closing already + client.disableWakeups(); + maybeCloseFetchSessions(timer); + super.closeInternal(timer); + } + + /** + * Creates the {@link FetchRequest.Builder fetch request}, + * {@link NetworkClient#send(ClientRequest, long) enqueues/sends it, and adds the {@link RequestFuture callback} + * for the response. + * + * @param fetchRequests {@link Map} of {@link Node nodes} to their + * {@link FetchSessionHandler.FetchRequestData request data} + * @param successHandler {@link ResponseHandler Handler for successful responses} + * @param errorHandler {@link ResponseHandler Handler for failure responses} + * @return List of {@link RequestFuture callbacks} + */ + private List> sendFetchesInternal(Map fetchRequests, + ResponseHandler successHandler, + ResponseHandler errorHandler) { + final List> requestFutures = new ArrayList<>(); + + for (Map.Entry entry : fetchRequests.entrySet()) { final Node fetchTarget = entry.getKey(); final FetchSessionHandler.FetchRequestData data = entry.getValue(); final FetchRequest.Builder request = createFetchRequest(fetchTarget, data); - RequestFutureListener listener = new RequestFutureListener() { + final RequestFuture responseFuture = client.send(fetchTarget, request); + + responseFuture.addListener(new RequestFutureListener() { @Override public void onSuccess(ClientResponse resp) { - synchronized (Fetcher.this) { - handleFetchResponse(fetchTarget, data, resp); - } + successHandler.handle(fetchTarget, data, resp); } @Override public void onFailure(RuntimeException e) { - synchronized (Fetcher.this) { - handleFetchResponse(fetchTarget, e); - } + errorHandler.handle(fetchTarget, data, e); } - }; + }); - final RequestFuture future = client.send(fetchTarget, request); - future.addListener(listener); + requestFutures.add(responseFuture); } - return fetchRequestMap.size(); - } - - public Fetch collectFetch() { - return fetchCollector.collectFetch(fetchBuffer); + return requestFutures; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index 2e2ed9f2ab4..2172313054d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -18,6 +18,8 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; @@ -58,6 +60,7 @@ import java.util.Collections; *

See {@link HeartbeatRequestState} for more details.

*/ public class HeartbeatRequestManager implements RequestManager { + private final Logger logger; private final Time time; @@ -90,22 +93,22 @@ public class HeartbeatRequestManager implements RequestManager { /** * ErrorEventHandler allows the background thread to propagate errors back to the user */ - private final ErrorEventHandler nonRetriableErrorHandler; + private final BackgroundEventHandler backgroundEventHandler; public HeartbeatRequestManager( - final Time time, final LogContext logContext, + final Time time, final ConsumerConfig config, final CoordinatorRequestManager coordinatorRequestManager, final SubscriptionState subscriptions, final MembershipManager membershipManager, - final ErrorEventHandler nonRetriableErrorHandler) { + final BackgroundEventHandler backgroundEventHandler) { this.coordinatorRequestManager = coordinatorRequestManager; this.time = time; this.logger = logContext.logger(getClass()); this.subscriptions = subscriptions; this.membershipManager = membershipManager; - this.nonRetriableErrorHandler = nonRetriableErrorHandler; + this.backgroundEventHandler = backgroundEventHandler; this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG); long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); @@ -122,7 +125,7 @@ public class HeartbeatRequestManager implements RequestManager { final SubscriptionState subscriptions, final MembershipManager membershipManager, final HeartbeatRequestState heartbeatRequestState, - final ErrorEventHandler nonRetriableErrorHandler) { + final BackgroundEventHandler backgroundEventHandler) { this.logger = logContext.logger(this.getClass()); this.time = time; this.subscriptions = subscriptions; @@ -130,7 +133,7 @@ public class HeartbeatRequestManager implements RequestManager { this.coordinatorRequestManager = coordinatorRequestManager; this.heartbeatRequestState = heartbeatRequestState; this.membershipManager = membershipManager; - this.nonRetriableErrorHandler = nonRetriableErrorHandler; + this.backgroundEventHandler = backgroundEventHandler; } /** @@ -146,18 +149,14 @@ public class HeartbeatRequestManager implements RequestManager { */ @Override public NetworkClientDelegate.PollResult poll(long currentTimeMs) { - if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat()) { - return new NetworkClientDelegate.PollResult( - Long.MAX_VALUE, Collections.emptyList()); - } + if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat()) + return NetworkClientDelegate.PollResult.EMPTY; // TODO: We will need to send a heartbeat response after partitions being revoke. This needs to be // implemented either with or after the partition reconciliation logic. - if (!heartbeatRequestState.canSendRequest(currentTimeMs)) { - return new NetworkClientDelegate.PollResult( - heartbeatRequestState.nextHeartbeatMs(currentTimeMs), - Collections.emptyList()); - } + if (!heartbeatRequestState.canSendRequest(currentTimeMs)) + return new NetworkClientDelegate.PollResult(heartbeatRequestState.nextHeartbeatMs(currentTimeMs)); + this.heartbeatRequestState.onSendAttempt(currentTimeMs); NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest(); return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request)); @@ -198,7 +197,7 @@ public class HeartbeatRequestManager implements RequestManager { this.heartbeatRequestState.onFailedAttempt(responseTimeMs); if (exception instanceof RetriableException) { String message = String.format("GroupHeartbeatRequest failed because of the retriable exception. " + - "Will retry in %s ms: {}", + "Will retry in %s ms: %s", heartbeatRequestState.remainingBackoffMs(responseTimeMs), exception.getMessage()); logger.debug(message); @@ -223,12 +222,13 @@ public class HeartbeatRequestManager implements RequestManager { final long currentTimeMs) { Errors error = Errors.forCode(response.data().errorCode()); String errorMessage = response.data().errorMessage(); + String message; // TODO: upon encountering a fatal/fenced error, trigger onPartitionLost logic to give up the current // assignments. switch (error) { case NOT_COORDINATOR: // the manager should retry immediately when the coordinator node becomes available again - String message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " + + message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " + "Will attempt to find the coordinator again and retry", coordinatorRequestManager.coordinator()); logInfo(message, response, currentTimeMs); @@ -274,7 +274,7 @@ public class HeartbeatRequestManager implements RequestManager { break; case FENCED_MEMBER_EPOCH: - message = String.format("GroupHeartbeatRequest failed because member epoch %s is invalid. " + + message = String.format("GroupHeartbeatRequest failed because member ID %s with epoch %s is invalid. " + "Will abandon all partitions and rejoin the group", membershipManager.memberId(), membershipManager.memberEpoch()); logInfo(message, response, currentTimeMs); @@ -282,7 +282,7 @@ public class HeartbeatRequestManager implements RequestManager { break; case UNKNOWN_MEMBER_ID: - message = String.format("GroupHeartbeatRequest failed because member id %s is invalid. " + + message = String.format("GroupHeartbeatRequest failed because member of unknown ID %s with epoch %s is invalid. " + "Will abandon all partitions and rejoin the group", membershipManager.memberId(), membershipManager.memberEpoch()); logInfo(message, response, currentTimeMs); @@ -307,7 +307,7 @@ public class HeartbeatRequestManager implements RequestManager { } private void handleFatalFailure(Throwable error) { - nonRetriableErrorHandler.handle(error); + backgroundEventHandler.add(new ErrorBackgroundEvent(error)); membershipManager.transitionToFailed(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 2e61b0788f8..47485ed46ca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.clients.RequestCompletionHandler; @@ -26,6 +28,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -35,27 +38,29 @@ import org.slf4j.Logger; import java.io.IOException; import java.util.ArrayDeque; import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Queue; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; +import java.util.function.Supplier; + +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; /** * A wrapper around the {@link org.apache.kafka.clients.NetworkClient} to handle network poll and send operations. */ public class NetworkClientDelegate implements AutoCloseable { + private final KafkaClient client; private final Time time; private final Logger log; private final int requestTimeoutMs; private final Queue unsentRequests; private final long retryBackoffMs; - private final Set tryConnectNodes; public NetworkClientDelegate( final Time time, @@ -68,9 +73,40 @@ public class NetworkClientDelegate implements AutoCloseable { this.unsentRequests = new ArrayDeque<>(); this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); - this.tryConnectNodes = new HashSet<>(); } + // Visible for testing + Queue unsentRequests() { + return unsentRequests; + } + + /** + * Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in + * reconnect backoff window following the disconnect). + * + * @param node {@link Node} to check for availability + * @see NetworkClientUtils#isUnavailable(KafkaClient, Node, Time) + */ + public boolean isUnavailable(Node node) { + return NetworkClientUtils.isUnavailable(client, node, time); + } + + /** + * Checks for an authentication error on a given node and throws the exception if it exists. + * + * @param node {@link Node} to check for a previous {@link AuthenticationException}; if found it is thrown + * @see NetworkClientUtils#maybeThrowAuthFailure(KafkaClient, Node) + */ + public void maybeThrowAuthFailure(Node node) { + NetworkClientUtils.maybeThrowAuthFailure(client, node); + } + + /** + * Initiate a connection if currently possible. This is only really useful for resetting + * the failed status of a socket. + * + * @param node The node to connect to + */ public void tryConnect(Node node) { NetworkClientUtils.tryConnect(client, node, time); } @@ -81,7 +117,6 @@ public class NetworkClientDelegate implements AutoCloseable { * * @param timeoutMs timeout time * @param currentTimeMs current time - * @return a list of client response */ public void poll(final long timeoutMs, final long currentTimeMs) { trySend(currentTimeMs); @@ -119,8 +154,7 @@ public class NetworkClientDelegate implements AutoCloseable { } } - private boolean doSend(final UnsentRequest r, - final long currentTimeMs) { + boolean doSend(final UnsentRequest r, final long currentTimeMs) { Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs)); if (node == null || nodeUnavailable(node)) { log.debug("No broker available to send the request: {}. Retrying.", r); @@ -137,7 +171,7 @@ public class NetworkClientDelegate implements AutoCloseable { return true; } - private void checkDisconnects(final long currentTimeMs) { + protected void checkDisconnects(final long currentTimeMs) { // Check the connection of the unsent request. Disconnect the disconnected node if it is unable to be connected. Iterator iter = unsentRequests.iterator(); while (iter.hasNext()) { @@ -190,26 +224,47 @@ public class NetworkClientDelegate implements AutoCloseable { this.client.close(); } + public long addAll(PollResult pollResult) { + addAll(pollResult.unsentRequests); + return pollResult.timeUntilNextPollMs; + } + public void addAll(final List requests) { - requests.forEach(u -> { - u.setTimer(this.time, this.requestTimeoutMs); - }); - this.unsentRequests.addAll(requests); + if (!requests.isEmpty()) { + requests.forEach(ur -> ur.setTimer(time, requestTimeoutMs)); + unsentRequests.addAll(requests); + } } public static class PollResult { + + public static final long WAIT_FOREVER = Long.MAX_VALUE; + public static final PollResult EMPTY = new PollResult(WAIT_FOREVER); public final long timeUntilNextPollMs; public final List unsentRequests; - public PollResult(final long timeMsTillNextPoll, final List unsentRequests) { - this.timeUntilNextPollMs = timeMsTillNextPoll; + public PollResult(final long timeUntilNextPollMs, final List unsentRequests) { + this.timeUntilNextPollMs = timeUntilNextPollMs; this.unsentRequests = Collections.unmodifiableList(unsentRequests); } + + public PollResult(final List unsentRequests) { + this(WAIT_FOREVER, unsentRequests); + } + + public PollResult(final UnsentRequest unsentRequest) { + this(Collections.singletonList(unsentRequest)); + } + + public PollResult(final long timeUntilNextPollMs) { + this(timeUntilNextPollMs, Collections.emptyList()); + } } + public static class UnsentRequest { private final AbstractRequest.Builder requestBuilder; private final FutureCompletionHandler handler; - private Optional node; // empty if random node can be chosen + private final Optional node; // empty if random node can be chosen private Timer timer; public UnsentRequest(final AbstractRequest.Builder requestBuilder, @@ -227,6 +282,12 @@ public class NetworkClientDelegate implements AutoCloseable { this.handler.future().whenComplete(callback); } + public UnsentRequest(final AbstractRequest.Builder requestBuilder, + final Node node, + final BiConsumer callback) { + this(requestBuilder, Optional.of(node), callback); + } + public void setTimer(final Time time, final long requestTimeoutMs) { this.timer = time.timer(requestTimeoutMs); } @@ -243,6 +304,10 @@ public class NetworkClientDelegate implements AutoCloseable { return requestBuilder; } + Optional node() { + return node; + } + @Override public String toString() { return "UnsentRequest{" + @@ -291,4 +356,32 @@ public class NetworkClientDelegate implements AutoCloseable { return future; } } + + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ + public static Supplier supplier(final Time time, + final LogContext logContext, + final ConsumerMetadata metadata, + final ConsumerConfig config, + final ApiVersions apiVersions, + final Metrics metrics, + final FetchMetricsManager fetchMetricsManager) { + return new CachedSupplier() { + @Override + protected NetworkClientDelegate create() { + KafkaClient client = ClientUtils.createNetworkClient(config, + metrics, + CONSUMER_METRIC_GROUP_PREFIX, + logContext, + apiVersions, + time, + CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION, + metadata, + fetchMetricsManager.throttleTimeSensor()); + return new NetworkClientDelegate(time, config, logContext, client); + } + }; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherUtils.java index b7fdefeb0d1..9239811f7d6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherUtils.java @@ -19,6 +19,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.LogTruncationException; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetResetStrategy; @@ -262,7 +263,7 @@ class OffsetFetcherUtils { else if (strategy == OffsetResetStrategy.LATEST) return ListOffsetsRequest.LATEST_TIMESTAMP; else - return null; + throw new NoOffsetForPartitionException(partition); } static Set topicsForPartitions(Collection partitions) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index a1cb8d54a97..700e2ab6e17 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -24,6 +24,8 @@ import org.apache.kafka.clients.consumer.LogTruncationException; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -83,6 +85,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis private final Time time; private final ApiVersions apiVersions; private final NetworkClientDelegate networkClientDelegate; + private final BackgroundEventHandler backgroundEventHandler; @SuppressWarnings("this-escape") public OffsetsRequestManager(final SubscriptionState subscriptionState, @@ -93,6 +96,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis final long requestTimeoutMs, final ApiVersions apiVersions, final NetworkClientDelegate networkClientDelegate, + final BackgroundEventHandler backgroundEventHandler, final LogContext logContext) { requireNonNull(subscriptionState); requireNonNull(metadata); @@ -100,6 +104,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis requireNonNull(time); requireNonNull(apiVersions); requireNonNull(networkClientDelegate); + requireNonNull(backgroundEventHandler); requireNonNull(logContext); this.metadata = metadata; @@ -112,11 +117,12 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis this.requestTimeoutMs = requestTimeoutMs; this.apiVersions = apiVersions; this.networkClientDelegate = networkClientDelegate; + this.backgroundEventHandler = backgroundEventHandler; this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState, time, retryBackoffMs, apiVersions); // Register the cluster metadata update callback. Note this only relies on the // requestsToRetry initialized above, and won't be invoked until all managers are - // initialized and the background thread started. + // initialized and the network thread started. this.metadata.addClusterUpdateListener(this); } @@ -127,10 +133,10 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis */ @Override public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { - NetworkClientDelegate.PollResult pollResult = - new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>(requestsToSend)); - this.requestsToSend.clear(); - return pollResult; + // Copy the outgoing request list and clear it. + List unsentRequests = new ArrayList<>(requestsToSend); + requestsToSend.clear(); + return new NetworkClientDelegate.PollResult(unsentRequests); } /** @@ -188,7 +194,14 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis * this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException}) */ public void resetPositionsIfNeeded() { - Map offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp(); + Map offsetResetTimestamps; + + try { + offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp(); + } catch (Exception e) { + backgroundEventHandler.add(new ErrorBackgroundEvent(e)); + return; + } if (offsetResetTimestamps.isEmpty()) return; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java index 195bee9ddd4..949616daa85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java @@ -20,25 +20,33 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerInterceptor; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.EventHandler; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; @@ -48,11 +56,12 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InvalidGroupIdException; -import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -67,121 +76,246 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Properties; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; import static java.util.Objects.requireNonNull; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.refreshCommittedOffsets; import static org.apache.kafka.common.utils.Utils.closeQuietly; import static org.apache.kafka.common.utils.Utils.isBlank; import static org.apache.kafka.common.utils.Utils.join; import static org.apache.kafka.common.utils.Utils.propsToMap; /** - * This prototype consumer uses the EventHandler to process application - * events so that the network IO can be processed in a background thread. Visit - * this document + * This prototype consumer uses an {@link ApplicationEventHandler event handler} to process + * {@link ApplicationEvent application events} so that the network IO can be processed in a dedicated + * {@link ConsumerNetworkThread network thread}. Visit + * this document * for detail implementation. */ public class PrototypeAsyncConsumer implements Consumer { - static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000; - private final LogContext logContext; - private final EventHandler eventHandler; + private final ApplicationEventHandler applicationEventHandler; private final Time time; private final Optional groupId; - private final Logger log; + private final KafkaConsumerMetrics kafkaConsumerMetrics; + private Logger log; + private final String clientId; + private final BackgroundEventProcessor backgroundEventProcessor; private final Deserializers deserializers; + + /** + * A thread-safe {@link FetchBuffer fetch buffer} for the results that are populated in the + * {@link ConsumerNetworkThread network thread} when the results are available. Because of the interaction + * of the fetch buffer in the application thread and the network I/O thread, this is shared between the + * two threads and is thus designed to be thread-safe. + */ + private final FetchBuffer fetchBuffer; + private final FetchCollector fetchCollector; + private final ConsumerInterceptors interceptors; + private final IsolationLevel isolationLevel; + private final SubscriptionState subscriptions; private final ConsumerMetadata metadata; private final Metrics metrics; + private final long retryBackoffMs; private final long defaultApiTimeoutMs; + private volatile boolean closed = false; + private final List assignors; + + // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates + private boolean cachedSubscriptionHasAllFetchPositions; + private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); - private WakeupTrigger wakeupTrigger = new WakeupTrigger(); - public PrototypeAsyncConsumer(Properties properties, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { + public PrototypeAsyncConsumer(final Properties properties, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer) { this(propsToMap(properties), keyDeserializer, valueDeserializer); } public PrototypeAsyncConsumer(final Map configs, - final Deserializer keyDeser, - final Deserializer valDeser) { - this(new ConsumerConfig(appendDeserializerToConfig(configs, keyDeser, valDeser)), keyDeser, valDeser); + final Deserializer keyDeserializer, + final Deserializer valueDeserializer) { + this(new ConsumerConfig(appendDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), + keyDeserializer, + valueDeserializer); } public PrototypeAsyncConsumer(final ConsumerConfig config, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { - this.time = Time.SYSTEM; - GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(config, - GroupRebalanceConfig.ProtocolType.CONSUMER); - this.groupId = Optional.ofNullable(groupRebalanceConfig.groupId); - this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); - this.logContext = createLogContext(config, groupRebalanceConfig); - this.log = logContext.logger(getClass()); - this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer); - this.subscriptions = createSubscriptionState(config, logContext); - this.metrics = createMetrics(config, time); - List> interceptorList = configuredConsumerInterceptors(config); - ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners( - metrics.reporters(), - interceptorList, - Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); - this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners); - final List addresses = ClientUtils.parseAndValidateAddresses(config); - metadata.bootstrap(addresses); - this.eventHandler = new DefaultEventHandler( - config, - groupRebalanceConfig, - logContext, - subscriptions, - new ApiVersions(), - this.metrics, - clusterResourceListeners, - null // this is coming from the fetcher, but we don't have one - ); + this(Time.SYSTEM, config, keyDeserializer, valueDeserializer); } - // Visible for testing - PrototypeAsyncConsumer(Time time, - LogContext logContext, - ConsumerConfig config, - SubscriptionState subscriptions, - ConsumerMetadata metadata, - EventHandler eventHandler, - Metrics metrics, - Optional groupId, - int defaultApiTimeoutMs) { - this.time = time; - this.logContext = logContext; + public PrototypeAsyncConsumer(final Time time, + final ConsumerConfig config, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer) { + try { + GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(config, + GroupRebalanceConfig.ProtocolType.CONSUMER); + + this.groupId = Optional.ofNullable(groupRebalanceConfig.groupId); + this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); + LogContext logContext = createLogContext(config, groupRebalanceConfig); + this.log = logContext.logger(getClass()); + groupId.ifPresent(groupIdStr -> { + if (groupIdStr.isEmpty()) { + log.warn("Support for using the empty group id by consumers is deprecated and will be removed in the next major release."); + } + }); + + log.debug("Initializing the Kafka consumer"); + this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); + this.time = time; + this.metrics = createMetrics(config, time); + this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + + List> interceptorList = configuredConsumerInterceptors(config); + this.interceptors = new ConsumerInterceptors<>(interceptorList); + this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer); + this.subscriptions = createSubscriptionState(config, logContext); + ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(), + interceptorList, + Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); + this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners); + final List addresses = ClientUtils.parseAndValidateAddresses(config); + metadata.bootstrap(addresses); + + FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics); + FetchConfig fetchConfig = new FetchConfig(config); + this.isolationLevel = fetchConfig.isolationLevel; + + ApiVersions apiVersions = new ApiVersions(); + final BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); + final BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + + // This FetchBuffer is shared between the application and network threads. + this.fetchBuffer = new FetchBuffer(logContext); + final Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier(time, + logContext, + metadata, + config, + apiVersions, + metrics, + fetchMetricsManager); + final Supplier requestManagersSupplier = RequestManagers.supplier(time, + logContext, + backgroundEventQueue, + metadata, + subscriptions, + fetchBuffer, + config, + groupRebalanceConfig, + apiVersions, + fetchMetricsManager, + networkClientDelegateSupplier); + final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, + metadata, + applicationEventQueue, + requestManagersSupplier); + this.applicationEventHandler = new ApplicationEventHandler(logContext, + time, + applicationEventQueue, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); + this.backgroundEventProcessor = new BackgroundEventProcessor(logContext, backgroundEventQueue); + this.assignors = ConsumerPartitionAssignor.getAssignorInstances( + config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) + ); + + // no coordinator will be constructed for the default (null) group id + if (!groupId.isPresent()) { + config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); + //config.ignore(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED); + } + + // The FetchCollector is only used on the application thread. + this.fetchCollector = new FetchCollector<>(logContext, + metadata, + subscriptions, + fetchConfig, + deserializers, + fetchMetricsManager, + time); + + this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); + + config.logUnused(); + AppInfoParser.registerAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics, time.milliseconds()); + log.debug("Kafka consumer initialized"); + } catch (Throwable t) { + // call close methods if internal objects are already constructed; this is to prevent resource leak. see KAFKA-2121 + // we do not need to call `close` at all when `log` is null, which means no internal objects were initialized. + if (this.log != null) { + close(Duration.ZERO, true); + } + // now propagate the exception + throw new KafkaException("Failed to construct kafka consumer", t); + } + } + + public PrototypeAsyncConsumer(LogContext logContext, + String clientId, + Deserializers deserializers, + FetchBuffer fetchBuffer, + FetchCollector fetchCollector, + ConsumerInterceptors interceptors, + Time time, + ApplicationEventHandler applicationEventHandler, + BlockingQueue backgroundEventQueue, + Metrics metrics, + SubscriptionState subscriptions, + ConsumerMetadata metadata, + long retryBackoffMs, + int defaultApiTimeoutMs, + List assignors, + String groupId) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; - this.metadata = metadata; + this.clientId = clientId; + this.fetchBuffer = fetchBuffer; + this.fetchCollector = fetchCollector; + this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; + this.interceptors = Objects.requireNonNull(interceptors); + this.time = time; + this.backgroundEventProcessor = new BackgroundEventProcessor(logContext, backgroundEventQueue); this.metrics = metrics; - this.groupId = groupId; + this.groupId = Optional.ofNullable(groupId); + this.metadata = metadata; + this.retryBackoffMs = retryBackoffMs; this.defaultApiTimeoutMs = defaultApiTimeoutMs; - this.deserializers = new Deserializers<>(config); - this.eventHandler = eventHandler; + this.deserializers = deserializers; + this.applicationEventHandler = applicationEventHandler; + this.assignors = assignors; + this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); } /** - * poll implementation using {@link EventHandler}. + * poll implementation using {@link ApplicationEventHandler}. * 1. Poll for background events. If there's a fetch response event, process the record and return it. If it is * another type of event, process it. * 2. Send fetches if needed. @@ -193,76 +327,33 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public ConsumerRecords poll(final Duration timeout) { Timer timer = time.timer(timeout); + try { - do { - if (!eventHandler.isEmpty()) { - final Optional backgroundEvent = eventHandler.poll(); - // processEvent() may process 3 types of event: - // 1. Errors - // 2. Callback Invocation - // 3. Fetch responses - // Errors will be handled or rethrown. - // Callback invocation will trigger callback function execution, which is blocking until completion. - // Successful fetch responses will be added to the completedFetches in the fetcher, which will then - // be processed in the collectFetches(). - backgroundEvent.ifPresent(event -> processEvent(event, timeout)); - } + kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs()); + + if (subscriptions.hasNoSubscriptionOrUserAssignment()) { + throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); + } - updateFetchPositionsIfNeeded(timer); + do { + updateAssignmentMetadataIfNeeded(timer); + final Fetch fetch = pollForFetches(timer); - // The idea here is to have the background thread sending fetches autonomously, and the fetcher - // uses the poll loop to retrieve successful fetchResponse and process them on the polling thread. - final Fetch fetch = collectFetches(); if (!fetch.isEmpty()) { - return processFetchResults(fetch); + if (fetch.records().isEmpty()) { + log.trace("Returning empty records from `poll()` " + + "since the consumer's position has advanced for at least one topic partition"); + } + + return interceptors.onConsume(new ConsumerRecords<>(fetch.records())); } // We will wait for retryBackoffMs - } while (time.timer(timeout).notExpired()); - } catch (final Exception e) { - throw new RuntimeException(e); - } - // TODO: Once we implement poll(), clear wakeupTrigger in a finally block: wakeupTrigger.clearActiveTask(); - - return ConsumerRecords.empty(); - } - - /** - * Set the fetch position to the committed position (if there is one) or reset it using the - * offset reset policy the user has configured (if partitions require reset) - * - * @return true if the operation completed without timing out - * @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 - */ - private boolean updateFetchPositionsIfNeeded(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. - ValidatePositionsApplicationEvent validatePositionsEvent = new ValidatePositionsApplicationEvent(); - eventHandler.add(validatePositionsEvent); - - // 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; - - // If there are partitions still needing a position and a reset policy is defined, - // request reset using the default policy. If no reset strategy is defined and there - // are partitions with a missing position, then we will raise a NoOffsetForPartitionException exception. - subscriptions.resetInitializingPositions(); + } while (timer.notExpired()); - // Reset positions using partition offsets retrieved from the leader, for any partitions - // which are awaiting reset. This will trigger a ListOffset request, retrieve the - // partition offsets according to the strategy (ex. earliest, latest), and update the - // positions. - ResetPositionsApplicationEvent resetPositionsEvent = new ResetPositionsApplicationEvent(); - eventHandler.add(resetPositionsEvent); - return true; + return ConsumerRecords.empty(); + } finally { + kafkaConsumerMetrics.recordPollEnd(timer.currentTimeMs()); + } } /** @@ -274,20 +365,6 @@ public class PrototypeAsyncConsumer implements Consumer { commitSync(Duration.ofMillis(defaultApiTimeoutMs)); } - private void processEvent(final BackgroundEvent backgroundEvent, final Duration timeout) { - // stubbed class - } - - private ConsumerRecords processFetchResults(final Fetch fetch) { - // stubbed class - return ConsumerRecords.empty(); - } - - private Fetch collectFetches() { - // stubbed class - return Fetch.empty(); - } - /** * This method sends a commit event to the EventHandler and return. */ @@ -324,50 +401,96 @@ public class PrototypeAsyncConsumer implements Consumer { // the task can only be woken up if the top level API call is commitSync wakeupTrigger.setActiveTask(commitEvent.future()); } - eventHandler.add(commitEvent); + applicationEventHandler.add(commitEvent); return commitEvent.future(); } @Override public void seek(TopicPartition partition, long offset) { - throw new KafkaException("method not implemented"); + if (offset < 0) + throw new IllegalArgumentException("seek offset must not be a negative number"); + + log.info("Seeking to offset {} for partition {}", offset, partition); + SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( + offset, + Optional.empty(), // This will ensure we skip validation + metadata.currentLeader(partition)); + subscriptions.seekUnvalidated(partition, newPosition); } @Override public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) { - throw new KafkaException("method not implemented"); + long offset = offsetAndMetadata.offset(); + if (offset < 0) { + throw new IllegalArgumentException("seek offset must not be a negative number"); + } + + if (offsetAndMetadata.leaderEpoch().isPresent()) { + log.info("Seeking to offset {} for partition {} with epoch {}", + offset, partition, offsetAndMetadata.leaderEpoch().get()); + } else { + log.info("Seeking to offset {} for partition {}", offset, partition); + } + Metadata.LeaderAndEpoch currentLeaderAndEpoch = metadata.currentLeader(partition); + SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition( + offsetAndMetadata.offset(), + offsetAndMetadata.leaderEpoch(), + currentLeaderAndEpoch); + updateLastSeenEpochIfNewer(partition, offsetAndMetadata); + subscriptions.seekUnvalidated(partition, newPosition); } @Override public void seekToBeginning(Collection partitions) { - throw new KafkaException("method not implemented"); + if (partitions == null) + throw new IllegalArgumentException("Partitions collection cannot be null"); + + Collection parts = partitions.isEmpty() ? subscriptions.assignedPartitions() : partitions; + subscriptions.requestOffsetReset(parts, OffsetResetStrategy.EARLIEST); } @Override public void seekToEnd(Collection partitions) { - throw new KafkaException("method not implemented"); + if (partitions == null) + throw new IllegalArgumentException("Partitions collection cannot be null"); + + Collection parts = partitions.isEmpty() ? subscriptions.assignedPartitions() : partitions; + subscriptions.requestOffsetReset(parts, OffsetResetStrategy.LATEST); } @Override public long position(TopicPartition partition) { - throw new KafkaException("method not implemented"); + return position(partition, Duration.ofMillis(defaultApiTimeoutMs)); } @Override public long position(TopicPartition partition, Duration timeout) { - throw new KafkaException("method not implemented"); + if (!subscriptions.isAssigned(partition)) + throw new IllegalStateException("You can only check the position for partitions assigned to this consumer."); + + Timer timer = time.timer(timeout); + do { + SubscriptionState.FetchPosition position = subscriptions.validPosition(partition); + if (position != null) + return position.offset; + + updateFetchPositions(timer); + } while (timer.notExpired()); + + throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the position " + + "for partition " + partition + " could be determined"); } @Override @Deprecated public OffsetAndMetadata committed(TopicPartition partition) { - throw new KafkaException("method not implemented"); + return committed(partition, Duration.ofMillis(defaultApiTimeoutMs)); } @Override @Deprecated public OffsetAndMetadata committed(TopicPartition partition, Duration timeout) { - throw new KafkaException("method not implemented"); + return committed(Collections.singleton(partition), timeout).get(partition); } @Override @@ -386,7 +509,7 @@ public class PrototypeAsyncConsumer implements Consumer { final OffsetFetchApplicationEvent event = new OffsetFetchApplicationEvent(partitions); wakeupTrigger.setActiveTask(event.future()); try { - return eventHandler.addAndGet(event, time.timer(timeout)); + return applicationEventHandler.addAndGet(event, time.timer(timeout)); } finally { wakeupTrigger.clearActiveTask(); } @@ -401,12 +524,12 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public Map metrics() { - throw new KafkaException("method not implemented"); + return Collections.unmodifiableMap(metrics.metrics()); } @Override public List partitionsFor(String topic) { - throw new KafkaException("method not implemented"); + return partitionsFor(topic, Duration.ofMillis(defaultApiTimeoutMs)); } @Override @@ -416,7 +539,7 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public Map> listTopics() { - throw new KafkaException("method not implemented"); + return listTopics(Duration.ofMillis(defaultApiTimeoutMs)); } @Override @@ -426,17 +549,23 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public Set paused() { - throw new KafkaException("method not implemented"); + return Collections.unmodifiableSet(subscriptions.pausedPartitions()); } @Override public void pause(Collection partitions) { - throw new KafkaException("method not implemented"); + log.debug("Pausing partitions {}", partitions); + for (TopicPartition partition: partitions) { + subscriptions.pause(partition); + } } @Override public void resume(Collection partitions) { - throw new KafkaException("method not implemented"); + log.debug("Resuming partitions {}", partitions); + for (TopicPartition partition: partitions) { + subscriptions.resume(partition); + } } @Override @@ -469,7 +598,7 @@ public class PrototypeAsyncConsumer implements Consumer { if (timeout.toMillis() == 0L) return listOffsetsEvent.emptyResult(); - return eventHandler.addAndGet(listOffsetsEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(listOffsetsEvent, time.timer(timeout)); } @Override @@ -502,20 +631,42 @@ public class PrototypeAsyncConsumer implements Consumer { if (partitions.isEmpty()) { return Collections.emptyMap(); } - Map timestampToSearch = - partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - final ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( + Map timestampToSearch = partitions + .stream() + .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( timestampToSearch, false); - Map offsetAndTimestampMap = - eventHandler.addAndGet(listOffsetsEvent, time.timer(timeout)); - return offsetAndTimestampMap.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), - e -> e.getValue().offset())); + Map offsetAndTimestampMap = applicationEventHandler.addAndGet( + listOffsetsEvent, + time.timer(timeout)); + return offsetAndTimestampMap + .entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); } @Override public OptionalLong currentLag(TopicPartition topicPartition) { - throw new KafkaException("method not implemented"); + final Long lag = subscriptions.partitionLag(topicPartition, isolationLevel); + + // if the log end offset is not known and hence cannot return lag and there is + // no in-flight list offset requested yet, + // issue a list offset request for that partition so that next time + // we may get the answer; we do not need to wait for the return value + // since we would not try to poll the network client synchronously + if (lag == null) { + if (subscriptions.partitionEndOffset(topicPartition, isolationLevel) == null && + !subscriptions.partitionEndOffsetRequested(topicPartition)) { + log.info("Requesting the log end offset for {} in order to compute lag", topicPartition); + subscriptions.requestPartitionEndOffset(topicPartition); + endOffsets(Collections.singleton(topicPartition), Duration.ofMillis(0)); + } + + return OptionalLong.empty(); + } + + return OptionalLong.of(lag); } @Override @@ -540,11 +691,37 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public void close(Duration timeout) { + if (timeout.toMillis() < 0) + throw new IllegalArgumentException("The timeout cannot be negative."); + + try { + if (!closed) { + // need to close before setting the flag since the close function + // itself may trigger rebalance callback that needs the consumer to be open still + close(timeout, false); + } + } finally { + closed = true; + } + } + + private void close(Duration timeout, boolean swallowException) { + log.trace("Closing the Kafka consumer"); AtomicReference firstException = new AtomicReference<>(); - closeQuietly(this.eventHandler, "event handler", firstException); + + if (applicationEventHandler != null) + closeQuietly(() -> applicationEventHandler.close(timeout), "Failed to close application event handler with a timeout(ms)=" + timeout, firstException); + + closeQuietly(fetchBuffer, "Failed to close the fetch buffer", firstException); + closeQuietly(interceptors, "consumer interceptors", firstException); + closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); + closeQuietly(metrics, "consumer metrics", firstException); + closeQuietly(deserializers, "consumer deserializers", firstException); + + AppInfoParser.unregisterAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics); log.debug("Kafka consumer has been closed"); Throwable exception = firstException.get(); - if (exception != null) { + if (exception != null && !swallowException) { if (exception instanceof InterruptException) { throw (InterruptException) exception; } @@ -575,19 +752,14 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public void commitSync(Map offsets, Duration timeout) { - CompletableFuture commitFuture = commit(offsets, true); + long commitStart = time.nanoseconds(); try { - commitFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS); - } catch (final TimeoutException e) { - throw new org.apache.kafka.common.errors.TimeoutException(e); - } catch (final InterruptedException e) { - throw new InterruptException(e); - } catch (final ExecutionException e) { - if (e.getCause() instanceof WakeupException) - throw new WakeupException(); - throw new KafkaException(e); + CompletableFuture commitFuture = commit(offsets, true); + offsets.forEach(this::updateLastSeenEpochIfNewer); + ConsumerUtils.getResult(commitFuture, time.timer(timeout)); } finally { wakeupTrigger.clearActiveTask(); + kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart); } } @@ -598,7 +770,7 @@ public class PrototypeAsyncConsumer implements Consumer { @Override public Set assignment() { - return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); + return Collections.unmodifiableSet(subscriptions.assignedPartitions()); } /** @@ -608,17 +780,43 @@ public class PrototypeAsyncConsumer implements Consumer { */ @Override public Set subscription() { - return Collections.unmodifiableSet(this.subscriptions.subscription()); + return Collections.unmodifiableSet(subscriptions.subscription()); } @Override public void subscribe(Collection topics) { - throw new KafkaException("method not implemented"); + subscribe(topics, new NoOpConsumerRebalanceListener()); } @Override public void subscribe(Collection topics, ConsumerRebalanceListener callback) { - throw new KafkaException("method not implemented"); + maybeThrowInvalidGroupIdException(); + if (topics == null) + throw new IllegalArgumentException("Topic collection to subscribe to cannot be null"); + if (topics.isEmpty()) { + // treat subscribing to empty topic list as the same as unsubscribing + unsubscribe(); + } else { + for (String topic : topics) { + if (isBlank(topic)) + throw new IllegalArgumentException("Topic collection to subscribe to cannot contain null or empty topic"); + } + + throwIfNoAssignorsConfigured(); + + // Clear the buffered data which are not a part of newly assigned topics + final Set currentTopicPartitions = new HashSet<>(); + + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (topics.contains(tp.topic())) + currentTopicPartitions.add(tp); + } + + fetchBuffer.retainAll(currentTopicPartitions); + log.info("Subscribed to topic(s): {}", join(topics, ", ")); + if (subscriptions.subscribe(new HashSet<>(topics), callback)) + metadata.requestUpdateForNewTopics(); + } } @Override @@ -628,8 +826,7 @@ public class PrototypeAsyncConsumer implements Consumer { } if (partitions.isEmpty()) { - // TODO: implementation of unsubscribe() will be included in forthcoming commits. - // this.unsubscribe(); + unsubscribe(); return; } @@ -639,38 +836,75 @@ public class PrototypeAsyncConsumer implements Consumer { 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 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. + * + *

+ * 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 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 poll(long timeout) { - throw new KafkaException("method not implemented"); + public ConsumerRecords poll(final long timeoutMs) { + return poll(Duration.ofMillis(timeoutMs)); } // Visible for testing @@ -678,17 +912,99 @@ public class PrototypeAsyncConsumer implements Consumer { return wakeupTrigger; } - private static ClusterResourceListeners configureClusterResourceListeners( - final Deserializer keyDeserializer, - final Deserializer valueDeserializer, - final List... candidateLists) { - ClusterResourceListeners clusterResourceListeners = new ClusterResourceListeners(); - for (List candidateList: candidateLists) - clusterResourceListeners.maybeAddAll(candidateList); + private Fetch pollForFetches(Timer timer) { + long pollTimeout = timer.remainingMs(); + + // if data is available already, return it immediately + final Fetch 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); + + // 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. + * + *

+ * + * This method will {@link ApplicationEventHandler#wakeupNetworkThread() wake up} the {@link ConsumerNetworkThread} before + * retuning. This is done as an optimization so that the next round of data can be pre-fetched. + */ + private Fetch collectFetch() { + final Fetch 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() && !initWithCommittedOffsetsIfNeeded(timer)) + return false; + + // If there are partitions still needing a position and a reset policy is defined, + // request reset using the default policy. If no reset strategy is defined and there + // are partitions with a missing position, then we will raise a NoOffsetForPartitionException exception. + subscriptions.resetInitializingPositions(); - clusterResourceListeners.maybeAdd(keyDeserializer); - clusterResourceListeners.maybeAdd(valueDeserializer); - return clusterResourceListeners; + // Reset positions using partition offsets retrieved from the leader, for any partitions + // which are awaiting reset. This will trigger a ListOffset request, retrieve the + // partition offsets according to the strategy (ex. earliest, latest), and update the + // positions. + applicationEventHandler.add(new ResetPositionsApplicationEvent()); + return true; } /** @@ -706,20 +1022,21 @@ public class PrototypeAsyncConsumer implements Consumer { * @param timer Timer bounding how long this method can block * @return true iff the operation completed within the timeout */ - private boolean refreshCommittedOffsetsIfNeeded(Timer timer) { + private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final Set initializingPartitions = subscriptions.initializingPartitions(); log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); try { - final Map offsets = eventHandler.addAndGet(new OffsetFetchApplicationEvent(initializingPartitions), timer); - return ConsumerUtils.refreshCommittedOffsets(offsets, this.metadata, this.subscriptions); - } catch (org.apache.kafka.common.errors.TimeoutException e) { + final OffsetFetchApplicationEvent event = new OffsetFetchApplicationEvent(initializingPartitions); + final Map offsets = applicationEventHandler.addAndGet(event, timer); + refreshCommittedOffsets(offsets, metadata, subscriptions); + return true; + } catch (TimeoutException e) { log.error("Couldn't refresh committed offsets before timeout expired"); return false; } } - // This is here temporary as we don't have public access to the ConsumerConfig in this module. public static Map appendDeserializerToConfig(Map configs, Deserializer keyDeserializer, @@ -737,6 +1054,17 @@ public class PrototypeAsyncConsumer implements Consumer { return newConfigs; } + private void throwIfNoAssignorsConfigured() { + if (assignors.isEmpty()) + throw new IllegalStateException("Must configure at least one partition assigner class name to " + + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); + } + + private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) { + if (offsetAndMetadata != null) + offsetAndMetadata.leaderEpoch().ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(topicPartition, epoch)); + } + private class DefaultOffsetCommitCallback implements OffsetCommitCallback { @Override public void onComplete(Map offsets, Exception exception) { @@ -744,4 +1072,13 @@ public class PrototypeAsyncConsumer implements Consumer { log.error("Offset commit with offsets {} failed", offsets, exception); } } + + boolean updateAssignmentMetadataIfNeeded(Timer timer) { + backgroundEventProcessor.process(); + + // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as + // in the previous implementation, because it will eventually involve group coordination + // logic + return updateFetchPositions(timer); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManager.java index 4655f5ef704..8592035dda2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManager.java @@ -16,13 +16,50 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; +import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY; + /** * {@code PollResult} consist of {@code UnsentRequest} if there are requests to send; otherwise, return the time till * the next poll event. */ public interface RequestManager { + + /** + * During normal operation of the {@link Consumer}, a request manager may need to send out network requests. + * Implementations can return {@link PollResult their need for network I/O} by returning the requests here. + * This method is called within a single-threaded context from + * {@link ConsumerNetworkThread the consumer's network I/O thread}. As such, there should be no need for + * synchronization protection in this method's implementation. + * + *

+ * + * Note: no network I/O occurs in this method. The method itself should not block for any reason. This + * method is called from the consumer's network I/O thread, so quick execution of this method in all + * request managers is critical to ensure that we can heartbeat in a timely fashion. + * + * @param currentTimeMs The current system time at which the method was called; useful for determining if + * time-sensitive operations should be performed + */ PollResult poll(long currentTimeMs); + /** + * On shutdown of the {@link Consumer}, a request manager may need to send out network requests. Implementations + * can signal that by returning the {@link PollResult close} requests here. Like {@link #poll(long)}, this method + * is called within a single-threaded context from {@link ConsumerNetworkThread the consumer's network I/O thread}. + * As such, there should be no need for synchronization protection in this method's implementation. + * + *

+ * + * Note: no network I/O occurs in this method. The method itself should not block for any reason. This + * method is called as an (indirect) result of {@link Consumer#close() the consumer's close method} being invoked. + * (Note that it is still invoked on the consumer's network I/O thread). Quick execution of this method in + * all request managers is critical to ensure that we can complete as many of the consumer's shutdown + * tasks as possible within the user-provided timeout. + */ + default PollResult pollOnClose() { + return EMPTY; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index 5a397532fe3..b600a528164 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -16,10 +16,25 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.common.internals.IdempotentCloser; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; + +import java.io.Closeable; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.function.Supplier; + +import static org.apache.kafka.common.utils.Utils.closeQuietly; import static java.util.Objects.requireNonNull; @@ -28,25 +43,31 @@ import static java.util.Objects.requireNonNull; * This allows callers to both use the specific {@link RequestManager} instance, or to iterate over the list via * the {@link #entries()} method. */ -public class RequestManagers { +public class RequestManagers implements Closeable { + private final Logger log; public final Optional coordinatorRequestManager; public final Optional commitRequestManager; - private final Optional heartbeatRequestManager; + public final Optional heartbeatRequestManager; public final OffsetsRequestManager offsetsRequestManager; public final TopicMetadataRequestManager topicMetadataRequestManager; + public final FetchRequestManager fetchRequestManager; private final List> entries; + private final IdempotentCloser closer = new IdempotentCloser(); - public RequestManagers(OffsetsRequestManager offsetsRequestManager, + public RequestManagers(LogContext logContext, + OffsetsRequestManager offsetsRequestManager, TopicMetadataRequestManager topicMetadataRequestManager, + FetchRequestManager fetchRequestManager, Optional coordinatorRequestManager, Optional commitRequestManager, Optional heartbeatRequestManager) { - this.offsetsRequestManager = requireNonNull(offsetsRequestManager, - "OffsetsRequestManager cannot be null"); + this.log = logContext.logger(RequestManagers.class); + this.offsetsRequestManager = requireNonNull(offsetsRequestManager, "OffsetsRequestManager cannot be null"); this.coordinatorRequestManager = coordinatorRequestManager; this.commitRequestManager = commitRequestManager; this.topicMetadataRequestManager = topicMetadataRequestManager; + this.fetchRequestManager = fetchRequestManager; this.heartbeatRequestManager = heartbeatRequestManager; List> list = new ArrayList<>(); @@ -55,10 +76,111 @@ public class RequestManagers { list.add(heartbeatRequestManager); list.add(Optional.of(offsetsRequestManager)); list.add(Optional.of(topicMetadataRequestManager)); + list.add(Optional.of(fetchRequestManager)); entries = Collections.unmodifiableList(list); } public List> entries() { return entries; } + + @Override + public void close() { + closer.close( + () -> { + log.debug("Closing RequestManagers"); + + entries.stream() + .filter(Optional::isPresent) + .map(Optional::get) + .filter(rm -> rm instanceof Closeable) + .map(rm -> (Closeable) rm) + .forEach(c -> closeQuietly(c, c.getClass().getSimpleName())); + log.debug("RequestManagers has been closed"); + }, + () -> log.debug("RequestManagers was already closed") + ); + } + + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ + public static Supplier supplier(final Time time, + final LogContext logContext, + final BlockingQueue backgroundEventQueue, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final FetchBuffer fetchBuffer, + final ConsumerConfig config, + final GroupRebalanceConfig groupRebalanceConfig, + final ApiVersions apiVersions, + final FetchMetricsManager fetchMetricsManager, + final Supplier networkClientDelegateSupplier) { + return new CachedSupplier() { + @Override + protected RequestManagers create() { + final NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get(); + final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); + final FetchConfig fetchConfig = new FetchConfig(config); + long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); + final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions, + metadata, + fetchConfig.isolationLevel, + time, + retryBackoffMs, + requestTimeoutMs, + apiVersions, + networkClientDelegate, + backgroundEventHandler, + logContext); + final FetchRequestManager fetch = new FetchRequestManager(logContext, + time, + metadata, + subscriptions, + fetchConfig, + fetchBuffer, + fetchMetricsManager, + networkClientDelegate); + final TopicMetadataRequestManager topic = new TopicMetadataRequestManager( + logContext, + config); + HeartbeatRequestManager heartbeatRequestManager = null; + CoordinatorRequestManager coordinator = null; + CommitRequestManager commit = null; + + if (groupRebalanceConfig != null && groupRebalanceConfig.groupId != null) { + final GroupState groupState = new GroupState(groupRebalanceConfig); + coordinator = new CoordinatorRequestManager(time, + logContext, + retryBackoffMs, + retryBackoffMaxMs, + backgroundEventHandler, + groupState.groupId); + commit = new CommitRequestManager(time, logContext, subscriptions, config, coordinator, groupState); + MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext); + heartbeatRequestManager = new HeartbeatRequestManager( + logContext, + time, + config, + coordinator, + subscriptions, + membershipManager, + backgroundEventHandler); + } + + return new RequestManagers( + logContext, + listOffsets, + topic, + fetch, + Optional.ofNullable(coordinator), + Optional.ofNullable(commit), + Optional.ofNullable(heartbeatRequestManager) + ); + } + }; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 2443e4e30e7..f62794280f4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -40,6 +40,8 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY; + /** *

* Manages the state of topic metadata requests. This manager returns a @@ -84,9 +86,7 @@ public class TopicMetadataRequestManager implements RequestManager { .filter(Optional::isPresent) .map(Optional::get) .collect(Collectors.toList()); - return requests.isEmpty() ? - new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>()) : - new NetworkClientDelegate.PollResult(0, Collections.unmodifiableList(requests)); + return requests.isEmpty() ? EMPTY : new NetworkClientDelegate.PollResult(0, requests); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index bf9bb1d4962..133836da3b7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -24,7 +24,7 @@ import java.util.Objects; public abstract class ApplicationEvent { public enum Type { - NOOP, COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, + COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java new file mode 100644 index 00000000000..2917d507d7f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -0,0 +1,115 @@ +/* + * 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.events; + +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; +import org.apache.kafka.clients.consumer.internals.RequestManagers; +import org.apache.kafka.common.internals.IdempotentCloser; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * An event handler that receives {@link ApplicationEvent application events} from the application thread which + * are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}. + */ +public class ApplicationEventHandler implements Closeable { + + private final Logger log; + private final BlockingQueue applicationEventQueue; + private final ConsumerNetworkThread networkThread; + private final IdempotentCloser closer = new IdempotentCloser(); + + public ApplicationEventHandler(final LogContext logContext, + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier) { + this.log = logContext.logger(ApplicationEventHandler.class); + this.applicationEventQueue = applicationEventQueue; + this.networkThread = new ConsumerNetworkThread(logContext, + time, + applicationEventProcessorSupplier, + networkClientDelegateSupplier, + requestManagersSupplier); + this.networkThread.start(); + } + + /** + * Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread} + * to alert the network I/O thread that it has something to process. + * + * @param event An {@link ApplicationEvent} created by the application thread + */ + public void add(final ApplicationEvent event) { + Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null"); + log.trace("Enqueued event: {}", event); + applicationEventQueue.add(event); + wakeupNetworkThread(); + } + + /** + * Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the next event(s) from the queue. + */ + public void wakeupNetworkThread() { + networkThread.wakeup(); + } + + /** + * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will + * return the result value upon successful completion; otherwise throws an error. + * + *

+ * + * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * + * @param event A {@link CompletableApplicationEvent} created by the polling thread + * @param timer Timer for which to wait for the event to complete + * @return Value that is the result of the event + * @param Type of return value of the event + */ + public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { + Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); + Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); + add(event); + return event.get(timer); + } + + @Override + public void close() { + close(Duration.ZERO); + } + + public void close(final Duration timeout) { + closer.close( + () -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"), + () -> log.warn("The application event handler was already closed") + ); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index f2f55860004..ccbdd21b9dc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -17,146 +17,188 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; -public class ApplicationEventProcessor { - - private final BlockingQueue backgroundEventQueue; +/** + * An {@link EventProcessor} that is created and executes in the {@link ConsumerNetworkThread network thread} + * which processes {@link ApplicationEvent application events} generated by the application thread. + */ +public class ApplicationEventProcessor extends EventProcessor { + private final Logger log; private final ConsumerMetadata metadata; - private final RequestManagers requestManagers; - public ApplicationEventProcessor(final BlockingQueue backgroundEventQueue, + public ApplicationEventProcessor(final LogContext logContext, + final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { - this.backgroundEventQueue = backgroundEventQueue; + super(logContext, applicationEventQueue); + this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; } - public boolean process(final ApplicationEvent event) { - Objects.requireNonNull(event); + /** + * Process the events—if any—that were produced by the application thread. It is possible that when processing + * an event generates an error. In such cases, the processor will log an exception, but we do not want those + * errors to be propagated to the caller. + */ + @Override + public void process() { + process((event, error) -> { }); + } + + @Override + public void process(ApplicationEvent event) { switch (event.type()) { - case NOOP: - return process((NoopApplicationEvent) event); case COMMIT: - return process((CommitApplicationEvent) event); + process((CommitApplicationEvent) event); + return; + case POLL: - return process((PollApplicationEvent) event); + process((PollApplicationEvent) event); + return; + case FETCH_COMMITTED_OFFSET: - return process((OffsetFetchApplicationEvent) event); + process((OffsetFetchApplicationEvent) event); + return; + case METADATA_UPDATE: - return process((NewTopicsMetadataUpdateRequestEvent) event); + process((NewTopicsMetadataUpdateRequestEvent) event); + return; + case ASSIGNMENT_CHANGE: - return process((AssignmentChangeApplicationEvent) event); + process((AssignmentChangeApplicationEvent) event); + return; + case TOPIC_METADATA: - return process((TopicMetadataApplicationEvent) event); + process((TopicMetadataApplicationEvent) event); + return; + case LIST_OFFSETS: - return process((ListOffsetsApplicationEvent) event); + process((ListOffsetsApplicationEvent) event); + return; + case RESET_POSITIONS: - return processResetPositionsEvent(); + processResetPositionsEvent(); + return; + case VALIDATE_POSITIONS: - return processValidatePositionsEvent(); + processValidatePositionsEvent(); + return; + + default: + log.warn("Application event type " + event.type() + " was not expected"); } - return false; } - /** - * Processes {@link NoopApplicationEvent} and enqueue a - * {@link NoopBackgroundEvent}. This is intentionally left here for - * demonstration purpose. - * - * @param event a {@link NoopApplicationEvent} - */ - private boolean process(final NoopApplicationEvent event) { - return backgroundEventQueue.add(new NoopBackgroundEvent(event.message())); + @Override + protected Class getEventClass() { + return ApplicationEvent.class; } - private boolean process(final PollApplicationEvent event) { + private void process(final PollApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { - return true; + return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); manager.updateAutoCommitTimer(event.pollTimeMs()); - return true; } - private boolean process(final CommitApplicationEvent event) { + private void process(final CommitApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { // Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id - // upfront so we should never get to this block. + // upfront, so we should never get to this block. Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set"); event.future().completeExceptionally(exception); - return false; + return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); event.chain(manager.addOffsetCommitRequest(event.offsets())); - return true; } - private boolean process(final OffsetFetchApplicationEvent event) { + private void process(final OffsetFetchApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { event.future().completeExceptionally(new KafkaException("Unable to fetch committed " + "offset because the CommittedRequestManager is not available. Check if group.id was set correctly")); - return false; + return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); event.chain(manager.addOffsetFetchRequest(event.partitions())); - return true; } - private boolean process(final NewTopicsMetadataUpdateRequestEvent event) { + private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { metadata.requestUpdateForNewTopics(); - return true; } - private boolean process(final AssignmentChangeApplicationEvent event) { + private void process(final AssignmentChangeApplicationEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { - return false; + return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); manager.updateAutoCommitTimer(event.currentTimeMs()); manager.maybeAutoCommit(event.offsets()); - return true; } - private boolean process(final ListOffsetsApplicationEvent event) { + private void process(final ListOffsetsApplicationEvent event) { final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), event.requireTimestamps()); event.chain(future); - return true; } - private boolean processResetPositionsEvent() { + private void processResetPositionsEvent() { requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); - return true; } - private boolean processValidatePositionsEvent() { + private void processValidatePositionsEvent() { requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); - return true; } - private boolean process(final TopicMetadataApplicationEvent event) { + private void process(final TopicMetadataApplicationEvent event) { final CompletableFuture>> future = - this.requestManagers.topicMetadataRequestManager.requestTopicMetadata(Optional.of(event.topic())); + this.requestManagers.topicMetadataRequestManager.requestTopicMetadata(Optional.of(event.topic())); event.chain(future); - return true; + } + + /** + * Creates a {@link Supplier} for deferred creation during invocation by + * {@link ConsumerNetworkThread}. + */ + public static Supplier supplier(final LogContext logContext, + final ConsumerMetadata metadata, + final BlockingQueue applicationEventQueue, + final Supplier requestManagersSupplier) { + return new CachedSupplier() { + @Override + protected ApplicationEventProcessor create() { + RequestManagers requestManagers = requestManagersSupplier.get(); + return new ApplicationEventProcessor( + logContext, + applicationEventQueue, + requestManagers, + metadata + ); + } + }; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index b0f7c3454f6..a7dc3e454a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -16,15 +16,17 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; + import java.util.Objects; /** - * This is the abstract definition of the events created by the background thread. + * This is the abstract definition of the events created by the {@link ConsumerNetworkThread network thread}. */ public abstract class BackgroundEvent { public enum Type { - NOOP, ERROR, + ERROR, } protected final Type type; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java new file mode 100644 index 00000000000..cafa426d033 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -0,0 +1,52 @@ +/* + * 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.events; + +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.Objects; +import java.util.Queue; + +/** + * An event handler that receives {@link BackgroundEvent background events} from the + * {@link ConsumerNetworkThread network thread} which are then made available to the application thread + * via the {@link BackgroundEventProcessor}. + */ + +public class BackgroundEventHandler { + + private final Logger log; + private final Queue backgroundEventQueue; + + public BackgroundEventHandler(final LogContext logContext, final Queue backgroundEventQueue) { + this.log = logContext.logger(BackgroundEventHandler.class); + this.backgroundEventQueue = backgroundEventQueue; + } + + /** + * Add a {@link BackgroundEvent} to the handler. + * + * @param event A {@link BackgroundEvent} created by the {@link ConsumerNetworkThread network thread} + */ + public void add(BackgroundEvent event) { + Objects.requireNonNull(event, "BackgroundEvent provided to add must be non-null"); + log.trace("Enqueued event: {}", event); + backgroundEventQueue.add(event); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java new file mode 100644 index 00000000000..cafd4fba492 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java @@ -0,0 +1,75 @@ +/* + * 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.events; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.LogContext; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.atomic.AtomicReference; + +/** + * An {@link EventProcessor} that is created and executes in the application thread for the purpose of processing + * {@link BackgroundEvent background events} generated by the {@link ConsumerNetworkThread network thread}. + * Those events are generally of two types: + * + *

    + *
  • Errors that occur in the network thread that need to be propagated to the application thread
  • + *
  • {@link ConsumerRebalanceListener} callbacks that are to be executed on the application thread
  • + *
+ */ +public class BackgroundEventProcessor extends EventProcessor { + + public BackgroundEventProcessor(final LogContext logContext, + final BlockingQueue backgroundEventQueue) { + super(logContext, backgroundEventQueue); + } + + /** + * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. + * It is possible that {@link ErrorBackgroundEvent an error} could occur when processing the events. + * In such cases, the processor will take a reference to the first error, continue to process the + * remaining events, and then throw the first error that occurred. + */ + @Override + public void process() { + AtomicReference firstError = new AtomicReference<>(); + process((event, error) -> firstError.compareAndSet(null, error)); + + if (firstError.get() != null) + throw firstError.get(); + } + + @Override + public void process(final BackgroundEvent event) { + if (event.type() == BackgroundEvent.Type.ERROR) + process((ErrorBackgroundEvent) event); + else + throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); + } + + @Override + protected Class getEventClass() { + return BackgroundEvent.class; + } + + private void process(final ErrorBackgroundEvent event) { + throw event.error(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 8146d9583ae..365c620e0c0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -27,7 +27,7 @@ import java.util.concurrent.CompletableFuture; * * @param */ -public abstract class CompletableApplicationEvent extends ApplicationEvent { +public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java new file mode 100644 index 00000000000..8fdcc20fa83 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -0,0 +1,25 @@ +/* + * 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.events; + +import java.util.concurrent.CompletableFuture; + +public interface CompletableEvent { + + CompletableFuture future(); + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java index 74d35983291..2945f22986b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java @@ -16,16 +16,18 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.KafkaException; + public class ErrorBackgroundEvent extends BackgroundEvent { - private final Throwable error; + private final RuntimeException error; - public ErrorBackgroundEvent(Throwable error) { + public ErrorBackgroundEvent(Throwable t) { super(Type.ERROR); - this.error = error; + this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); } - public Throwable error() { + public RuntimeException error() { return error; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java deleted file mode 100644 index 75075fc2779..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.events; - -import org.apache.kafka.common.utils.Timer; - -import java.io.Closeable; -import java.util.Optional; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -/** - * This class interfaces with the KafkaConsumer and the background thread. It allows the caller to enqueue events via - * the {@code add()} method and to retrieve events via the {@code poll()} method. - */ -public interface EventHandler extends Closeable { - /** - * Retrieves and removes a {@link BackgroundEvent}. Returns an empty Optional instance if there is nothing. - * @return an Optional of {@link BackgroundEvent} if the value is present. Otherwise, an empty Optional. - */ - Optional poll(); - - /** - * Check whether there are pending {@code BackgroundEvent} await to be consumed. - * @return true if there are no pending event - */ - boolean isEmpty(); - - /** - * Add an {@link ApplicationEvent} to the handler. The method returns true upon successful add; otherwise returns - * false. - * @param event An {@link ApplicationEvent} created by the polling thread. - * @return true upon successful add. - */ - boolean add(ApplicationEvent event); - - /** - * Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will - * return the result value upon successful completion; otherwise throws an error. - * - *

- * - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. - * - * @param event A {@link CompletableApplicationEvent} created by the polling thread. - * @param timer Timer for which to wait for the event to complete - * @return Value that is the result of the event - * @param Type of return value of the event - */ - T addAndGet(final CompletableApplicationEvent event, final Timer timer); -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java new file mode 100644 index 00000000000..06e7ec28dd7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -0,0 +1,137 @@ +/* + * 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.events; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.internals.IdempotentCloser; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.io.Closeable; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.BlockingQueue; + +/** + * An {@link EventProcessor} is the means by which events produced by thread A are + * processed by thread B. By definition, threads A and B run in parallel to + * each other, so a mechanism is needed with which to receive and process the events from the other thread. That + * communication channel is formed around {@link BlockingQueue a shared queue} into which thread A + * enqueues events and thread B reads and processes those events. + */ +public abstract class EventProcessor implements Closeable { + + private final Logger log; + private final BlockingQueue eventQueue; + private final IdempotentCloser closer; + + protected EventProcessor(final LogContext logContext, final BlockingQueue eventQueue) { + this.log = logContext.logger(EventProcessor.class); + this.eventQueue = eventQueue; + this.closer = new IdempotentCloser(); + } + + public abstract void process(); + + public abstract void process(T event); + + @Override + public void close() { + closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); + } + + protected abstract Class getEventClass(); + + protected interface ProcessErrorHandler { + + void onError(T event, KafkaException error); + } + + /** + * Drains all available events from the queue, and then processes them in order. If any errors are thrown while + * processing the individual events, these are submitted to the given {@link ProcessErrorHandler}. + */ + protected void process(ProcessErrorHandler processErrorHandler) { + String eventClassName = getEventClass().getSimpleName(); + closer.assertOpen(() -> String.format("The processor was previously closed, so no further %s processing can occur", eventClassName)); + + List events = drain(); + + try { + log.debug("Starting processing of {} {}(s)", events.size(), eventClassName); + + for (T event : events) { + try { + Objects.requireNonNull(event, () -> String.format("Attempted to process a null %s", eventClassName)); + log.debug("Consuming {}: {}", eventClassName, event); + process(event); + } catch (Throwable t) { + log.warn("An error occurred when processing the {}: {}", eventClassName, t.getMessage(), t); + + KafkaException error; + + if (t instanceof KafkaException) + error = (KafkaException) t; + else + error = new KafkaException(t); + + processErrorHandler.onError(event, error); + } + } + } finally { + log.debug("Completed processing of {} {}(s)", events.size(), eventClassName); + } + } + + /** + * It is possible for the consumer to close before complete processing all the events in the queue. In + * this case, we need to throw an exception to notify the user the consumer is closed. + */ + private void closeInternal() { + String eventClassName = getEventClass().getSimpleName(); + log.trace("Closing event processor for {}", eventClassName); + List incompleteEvents = drain(); + + if (incompleteEvents.isEmpty()) + return; + + KafkaException exception = new KafkaException("The consumer is closed"); + + // Check each of the events and if it has a Future that is incomplete, complete it exceptionally. + incompleteEvents + .stream() + .filter(e -> e instanceof CompletableEvent) + .map(e -> ((CompletableEvent) e).future()) + .filter(f -> !f.isDone()) + .forEach(f -> { + log.debug("Completing {} with exception {}", f, exception.getMessage()); + f.completeExceptionally(exception); + }); + + log.debug("Discarding {} {}s because the consumer is closing", incompleteEvents.size(), eventClassName); + } + + /** + * Moves all the events from the queue to the returned list. + */ + private List drain() { + LinkedList events = new LinkedList<>(); + eventQueue.drainTo(events); + return events; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopApplicationEvent.java deleted file mode 100644 index 22817fb2bae..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopApplicationEvent.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.events; - -import java.util.Objects; - -/** - * The event is a no-op, but is intentionally left here for demonstration and test purposes. - */ -public class NoopApplicationEvent extends ApplicationEvent { - - private final String message; - - public NoopApplicationEvent(final String message) { - super(Type.NOOP); - this.message = Objects.requireNonNull(message); - } - - public String message() { - return message; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - NoopApplicationEvent that = (NoopApplicationEvent) o; - - return message.equals(that.message); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + message.hashCode(); - return result; - } - - @Override - public String toString() { - return "NoopApplicationEvent{" + - toStringBase() + - ",message='" + message + '\'' + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopBackgroundEvent.java deleted file mode 100644 index c1cbcc253a9..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NoopBackgroundEvent.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.events; - -import java.util.Objects; - -/** - * No-op event. Intentionally left it here for demonstration purpose. - */ -public class NoopBackgroundEvent extends BackgroundEvent { - - private final String message; - - public NoopBackgroundEvent(final String message) { - super(Type.NOOP); - this.message = Objects.requireNonNull(message); - } - - public String message() { - return message; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - NoopBackgroundEvent that = (NoopBackgroundEvent) o; - - return message.equals(that.message); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + message.hashCode(); - return result; - } - - @Override - public String toString() { - return "NoopBackgroundEvent{" + - toStringBase() + - ", message='" + message + '\'' + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index f61d4547d8b..6a46b453191 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -2674,7 +2674,7 @@ public class KafkaConsumerTest { } IsolationLevel isolationLevel = IsolationLevel.READ_UNCOMMITTED; FetchMetricsManager metricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics); - FetchConfig fetchConfig = new FetchConfig<>( + FetchConfig fetchConfig = new FetchConfig( minBytes, maxBytes, maxWaitMs, @@ -2682,7 +2682,6 @@ public class KafkaConsumerTest { maxPollRecords, checkCrcs, CommonClientConfigs.DEFAULT_CLIENT_RACK, - new Deserializers<>(keyDeserializer, deserializer), isolationLevel); Fetcher fetcher = new Fetcher<>( loggerFactory, @@ -2690,6 +2689,7 @@ public class KafkaConsumerTest { metadata, subscription, fetchConfig, + new Deserializers<>(keyDeserializer, deserializer), metricsManager, time); OffsetFetcher offsetFetcher = new OffsetFetcher(loggerFactory, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index c8a2e1ee945..bb58c9423c5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -55,6 +55,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_COMMIT_INTER import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -76,7 +77,7 @@ public class CommitRequestManagerTest { this.time = new MockTime(0); this.subscriptionState = mock(SubscriptionState.class); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); - this.groupState = new GroupState("group-1", Optional.empty()); + this.groupState = new GroupState(DEFAULT_GROUP_ID, Optional.empty()); this.props = new Properties(); this.props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 100); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java index de97ee40de2..2c39f4411fe 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java @@ -34,7 +34,6 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.UUIDDeserializer; import org.apache.kafka.common.serialization.UUIDSerializer; @@ -67,23 +66,22 @@ public class CompletedFetchTest { FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() .setRecords(newRecords(startingOffset, numRecords, fetchOffset)); - FetchConfig fetchConfig = newFetchConfig(new StringDeserializer(), - new StringDeserializer(), - IsolationLevel.READ_UNCOMMITTED, - true); + Deserializers deserializers = newStringDeserializers(); + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true); + CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData); - List> records = completedFetch.fetchRecords(fetchConfig, 10); + List> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(10, records.size()); ConsumerRecord record = records.get(0); assertEquals(10, record.offset()); - records = completedFetch.fetchRecords(fetchConfig, 10); + records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(1, records.size()); record = records.get(0); assertEquals(20, record.offset()); - records = completedFetch.fetchRecords(fetchConfig, 10); + records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(0, records.size()); } @@ -96,21 +94,15 @@ public class CompletedFetchTest { .setRecords(rawRecords) .setAbortedTransactions(newAbortedTransactions()); - try (final StringDeserializer deserializer = new StringDeserializer()) { - FetchConfig fetchConfig = newFetchConfig(deserializer, - deserializer, - IsolationLevel.READ_COMMITTED, - true); + try (final Deserializers deserializers = newStringDeserializers()) { + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, true); CompletedFetch completedFetch = newCompletedFetch(0, partitionData); - List> records = completedFetch.fetchRecords(fetchConfig, 10); + List> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(0, records.size()); - fetchConfig = newFetchConfig(deserializer, - deserializer, - IsolationLevel.READ_UNCOMMITTED, - true); + fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true); completedFetch = newCompletedFetch(0, partitionData); - records = completedFetch.fetchRecords(fetchConfig, 10); + records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(numRecords, records.size()); } } @@ -122,12 +114,9 @@ public class CompletedFetchTest { FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() .setRecords(rawRecords); CompletedFetch completedFetch = newCompletedFetch(0, partitionData); - try (final StringDeserializer deserializer = new StringDeserializer()) { - FetchConfig fetchConfig = newFetchConfig(deserializer, - deserializer, - IsolationLevel.READ_COMMITTED, - true); - List> records = completedFetch.fetchRecords(fetchConfig, 10); + try (final Deserializers deserializers = newStringDeserializers()) { + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, true); + List> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(10, records.size()); } } @@ -140,14 +129,13 @@ public class CompletedFetchTest { FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() .setRecords(newRecords(startingOffset, numRecords, fetchOffset)); - CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData); - FetchConfig fetchConfig = newFetchConfig(new StringDeserializer(), - new StringDeserializer(), - IsolationLevel.READ_UNCOMMITTED, - true); + try (final Deserializers deserializers = newStringDeserializers()) { + CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData); + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true); - List> records = completedFetch.fetchRecords(fetchConfig, -10); - assertEquals(0, records.size()); + List> records = completedFetch.fetchRecords(fetchConfig, deserializers, -10); + assertEquals(0, records.size()); + } } @Test @@ -159,13 +147,9 @@ public class CompletedFetchTest { .setLogStartOffset(0); CompletedFetch completedFetch = newCompletedFetch(1, partitionData); - try (final StringDeserializer deserializer = new StringDeserializer()) { - FetchConfig fetchConfig = newFetchConfig(deserializer, - deserializer, - IsolationLevel.READ_UNCOMMITTED, - true); - - List> records = completedFetch.fetchRecords(fetchConfig, 10); + try (final Deserializers deserializers = newStringDeserializers()) { + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true); + List> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10); assertEquals(0, records.size()); } } @@ -174,8 +158,7 @@ public class CompletedFetchTest { public void testCorruptedMessage() { // Create one good record and then one "corrupted" record. try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0); - final UUIDSerializer serializer = new UUIDSerializer(); - final UUIDDeserializer deserializer = new UUIDDeserializer()) { + final UUIDSerializer serializer = new UUIDSerializer()) { builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); builder.append(0L, "key".getBytes(), "value".getBytes()); Records records = builder.build(); @@ -187,16 +170,15 @@ public class CompletedFetchTest { .setLogStartOffset(0) .setRecords(records); - FetchConfig fetchConfig = newFetchConfig(deserializer, - deserializer, - IsolationLevel.READ_COMMITTED, - false); - CompletedFetch completedFetch = newCompletedFetch(0, partitionData); + try (final Deserializers deserializers = newUuidDeserializers()) { + FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, false); + CompletedFetch completedFetch = newCompletedFetch(0, partitionData); - completedFetch.fetchRecords(fetchConfig, 10); + completedFetch.fetchRecords(fetchConfig, deserializers, 10); - assertThrows(RecordDeserializationException.class, - () -> completedFetch.fetchRecords(fetchConfig, 10)); + assertThrows(RecordDeserializationException.class, + () -> completedFetch.fetchRecords(fetchConfig, deserializers, 10)); + } } } @@ -219,11 +201,16 @@ public class CompletedFetchTest { ApiKeys.FETCH.latestVersion()); } - private static FetchConfig newFetchConfig(Deserializer keyDeserializer, - Deserializer valueDeserializer, - IsolationLevel isolationLevel, - boolean checkCrcs) { - return new FetchConfig<>( + private static Deserializers newUuidDeserializers() { + return new Deserializers<>(new UUIDDeserializer(), new UUIDDeserializer()); + } + + private static Deserializers newStringDeserializers() { + return new Deserializers<>(new StringDeserializer(), new StringDeserializer()); + } + + private static FetchConfig newFetchConfig(IsolationLevel isolationLevel, boolean checkCrcs) { + return new FetchConfig( ConsumerConfig.DEFAULT_FETCH_MIN_BYTES, ConsumerConfig.DEFAULT_FETCH_MAX_BYTES, ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS, @@ -231,7 +218,6 @@ public class CompletedFetchTest { ConsumerConfig.DEFAULT_MAX_POLL_RECORDS, checkCrcs, ConsumerConfig.DEFAULT_CLIENT_RACK, - new Deserializers<>(keyDeserializer, valueDeserializer), isolationLevel ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 43ceca65bb7..cc0b1294fc2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -3082,7 +3082,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertTrue(subscriptions.hasAllFetchPositions()); @@ -3103,7 +3103,7 @@ public abstract class ConsumerCoordinatorTest { // Load offsets from previous epoch client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L, Optional.of(3))); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); // Offset gets loaded, but requires validation assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); @@ -3155,7 +3155,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertTrue(subscriptions.hasAllFetchPositions()); @@ -3170,7 +3170,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap())); try { - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); fail("Expected group authorization error"); } catch (GroupAuthorizationException e) { assertEquals(groupId, e.groupId()); @@ -3186,9 +3186,9 @@ public abstract class ConsumerCoordinatorTest { client.prepareResponse(offsetFetchResponse(t1p, Errors.UNSTABLE_OFFSET_COMMIT, "", -1L)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions()); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(0L)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(0L)); assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions()); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(0L)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(0L)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertTrue(subscriptions.hasAllFetchPositions()); @@ -3202,7 +3202,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L)); - assertThrows(KafkaException.class, () -> coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE))); + assertThrows(KafkaException.class, () -> coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE))); } @Test @@ -3214,7 +3214,7 @@ public abstract class ConsumerCoordinatorTest { client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertTrue(subscriptions.hasAllFetchPositions()); @@ -3228,7 +3228,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", -1L)); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions()); assertEquals(Collections.emptySet(), subscriptions.partitionsNeedingReset(time.milliseconds())); @@ -3242,7 +3242,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); subscriptions.seek(t1p, 500L); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertTrue(subscriptions.hasAllFetchPositions()); @@ -3256,7 +3256,7 @@ public abstract class ConsumerCoordinatorTest { subscriptions.assignFromUser(singleton(t1p)); subscriptions.requestOffsetReset(t1p, OffsetResetStrategy.EARLIEST); - coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); + coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertFalse(subscriptions.hasAllFetchPositions()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java new file mode 100644 index 00000000000..812999b3ca3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -0,0 +1,269 @@ +/* + * 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.MockClient; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.RequestTestUtils; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.test.TestCondition; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; + +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; +import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@SuppressWarnings("ClassDataAbstractionCoupling") +public class ConsumerNetworkThreadTest { + + private ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder testBuilder; + private Time time; + private ConsumerMetadata metadata; + private NetworkClientDelegate networkClient; + private BlockingQueue applicationEventsQueue; + private ApplicationEventProcessor applicationEventProcessor; + private OffsetsRequestManager offsetsRequestManager; + private CommitRequestManager commitManager; + private ConsumerNetworkThread consumerNetworkThread; + private MockClient client; + + @BeforeEach + public void setup() { + testBuilder = new ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder(); + time = testBuilder.time; + metadata = testBuilder.metadata; + networkClient = testBuilder.networkClientDelegate; + client = testBuilder.client; + applicationEventsQueue = testBuilder.applicationEventQueue; + applicationEventProcessor = testBuilder.applicationEventProcessor; + commitManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); + offsetsRequestManager = testBuilder.offsetsRequestManager; + consumerNetworkThread = testBuilder.consumerNetworkThread; + consumerNetworkThread.initializeResources(); + } + + @AfterEach + public void tearDown() { + if (testBuilder != null) + testBuilder.close(); + } + + @Test + public void testStartupAndTearDown() throws InterruptedException { + // The consumer is closed in ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder.close() + // which is called from tearDown(). + consumerNetworkThread.start(); + + TestCondition isStarted = () -> consumerNetworkThread.isRunning(); + TestCondition isClosed = () -> !(consumerNetworkThread.isRunning() || consumerNetworkThread.isAlive()); + + // There's a nonzero amount of time between starting the thread and having it + // begin to execute our code. Wait for a bit before checking... + TestUtils.waitForCondition(isStarted, + "The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms"); + + consumerNetworkThread.close(Duration.ofMillis(DEFAULT_MAX_WAIT_MS)); + + TestUtils.waitForCondition(isClosed, + "The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms"); + } + + @Test + public void testApplicationEvent() { + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>()); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor, times(1)).process(e); + } + + @Test + public void testMetadataUpdateEvent() { + ApplicationEvent e = new NewTopicsMetadataUpdateRequestEvent(); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(metadata).requestUpdateForNewTopics(); + } + + @Test + public void testCommitEvent() { + ApplicationEvent e = new CommitApplicationEvent(new HashMap<>()); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(CommitApplicationEvent.class)); + } + + @Test + public void testListOffsetsEventIsProcessed() { + Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); + ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); + assertTrue(applicationEventsQueue.isEmpty()); + } + + @Test + public void testResetPositionsEventIsProcessed() { + ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + assertTrue(applicationEventsQueue.isEmpty()); + } + + @Test + public void testResetPositionsProcessFailureIsIgnored() { + doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); + + ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); + applicationEventsQueue.add(event); + assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); + + verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + } + + @Test + public void testValidatePositionsEventIsProcessed() { + ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); + applicationEventsQueue.add(e); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); + assertTrue(applicationEventsQueue.isEmpty()); + } + + @Test + public void testAssignmentChangeEvent() { + HashMap offset = mockTopicPartitionOffset(); + + final long currentTimeMs = time.milliseconds(); + ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs); + applicationEventsQueue.add(e); + + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class)); + verify(networkClient, times(1)).poll(anyLong(), anyLong()); + verify(commitManager, times(1)).updateAutoCommitTimer(currentTimeMs); + verify(commitManager, times(1)).maybeAutoCommit(offset); + } + + @Test + void testFetchTopicMetadata() { + applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic")); + consumerNetworkThread.runOnce(); + verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); + } + + @Test + void testPollResultTimer() { + NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) + .setKey("foobar")), + Optional.empty()); + req.setTimer(time, DEFAULT_REQUEST_TIMEOUT_MS); + + // purposely setting a non-MAX time to ensure it is returning Long.MAX_VALUE upon success + NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult( + 10, + Collections.singletonList(req)); + assertEquals(10, networkClient.addAll(success)); + + NetworkClientDelegate.PollResult failure = new NetworkClientDelegate.PollResult( + 10, + new ArrayList<>()); + assertEquals(10, networkClient.addAll(failure)); + } + + @Test + void testRequestManagersArePolledOnce() { + consumerNetworkThread.runOnce(); + testBuilder.requestManagers.entries().forEach(rmo -> rmo.ifPresent(rm -> verify(rm, times(1)).poll(anyLong()))); + verify(networkClient, times(1)).poll(anyLong(), anyLong()); + } + + @Test + void testEnsureMetadataUpdateOnPoll() { + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); + client.prepareMetadataUpdate(metadataResponse); + metadata.requestUpdate(false); + consumerNetworkThread.runOnce(); + verify(metadata, times(1)).updateWithCurrentRequestVersion(eq(metadataResponse), eq(false), anyLong()); + } + + @Test + void testEnsureEventsAreCompleted() { + CompletableApplicationEvent event1 = spy(new CommitApplicationEvent(Collections.emptyMap())); + ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap()); + CompletableFuture future = new CompletableFuture<>(); + when(event1.future()).thenReturn(future); + applicationEventsQueue.add(event1); + applicationEventsQueue.add(event2); + assertFalse(future.isDone()); + assertFalse(applicationEventsQueue.isEmpty()); + + consumerNetworkThread.cleanup(); + assertTrue(future.isCompletedExceptionally()); + assertTrue(applicationEventsQueue.isEmpty()); + } + + private HashMap mockTopicPartitionOffset() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); + return topicPartitionOffsets; + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java new file mode 100644 index 00000000000..894f8305d02 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -0,0 +1,370 @@ +/* + * 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.CommonClientConfigs; +import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor; +import org.apache.kafka.common.internals.ClusterResourceListeners; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.RequestTestUtils; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; + +import java.io.Closeable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_INSTANCE_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; +import static org.apache.kafka.common.utils.Utils.closeQuietly; +import static org.mockito.Mockito.spy; + +@SuppressWarnings("ClassDataAbstractionCoupling") +public class ConsumerTestBuilder implements Closeable { + + static final long DEFAULT_RETRY_BACKOFF_MS = 80; + static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; + static final int DEFAULT_REQUEST_TIMEOUT_MS = 500; + static final int DEFAULT_MAX_POLL_INTERVAL_MS = 10000; + static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id"; + static final String DEFAULT_GROUP_ID = "group-id"; + static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; + static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; + + final LogContext logContext = new LogContext(); + final Time time = new MockTime(0); + public final BlockingQueue applicationEventQueue; + public final BlockingQueue backgroundEventQueue; + final ConsumerConfig config; + final long retryBackoffMs; + final SubscriptionState subscriptions; + final ConsumerMetadata metadata; + final FetchConfig fetchConfig; + final FetchBuffer fetchBuffer; + final Metrics metrics; + final FetchMetricsManager metricsManager; + final NetworkClientDelegate networkClientDelegate; + final OffsetsRequestManager offsetsRequestManager; + final Optional coordinatorRequestManager; + final Optional commitRequestManager; + final Optional heartbeatRequestManager; + final Optional membershipManager; + final Optional heartbeatRequestState; + final TopicMetadataRequestManager topicMetadataRequestManager; + final FetchRequestManager fetchRequestManager; + final RequestManagers requestManagers; + public final ApplicationEventProcessor applicationEventProcessor; + public final BackgroundEventProcessor backgroundEventProcessor; + public final BackgroundEventHandler backgroundEventHandler; + final MockClient client; + final Optional groupInfo; + + public ConsumerTestBuilder() { + this(Optional.empty()); + } + + public ConsumerTestBuilder(Optional groupInfo) { + this.groupInfo = groupInfo; + this.applicationEventQueue = new LinkedBlockingQueue<>(); + this.backgroundEventQueue = new LinkedBlockingQueue<>(); + this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue)); + GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( + 100, + DEFAULT_MAX_POLL_INTERVAL_MS, + DEFAULT_HEARTBEAT_INTERVAL_MS, + groupInfo.map(gi -> gi.groupState.groupId).orElse(null), + groupInfo.flatMap(gi -> gi.groupState.groupInstanceId), + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + true); + GroupState groupState = new GroupState(groupRebalanceConfig); + ApiVersions apiVersions = new ApiVersions(); + + Properties properties = new Properties(); + properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, DEFAULT_RETRY_BACKOFF_MS); + properties.put(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, DEFAULT_REQUEST_TIMEOUT_MS); + properties.put(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG, DEFAULT_MAX_POLL_INTERVAL_MS); + + groupInfo.ifPresent(gi -> { + properties.put(GROUP_ID_CONFIG, gi.groupState.groupId); + gi.groupState.groupInstanceId.ifPresent(groupInstanceId -> properties.put(GROUP_INSTANCE_ID_CONFIG, groupInstanceId)); + }); + + this.config = new ConsumerConfig(properties); + + this.fetchConfig = new FetchConfig(config); + this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + this.metrics = createMetrics(config, time); + + this.subscriptions = spy(createSubscriptionState(config, logContext)); + this.metadata = spy(new ConsumerMetadata(config, subscriptions, logContext, new ClusterResourceListeners())); + this.metricsManager = createFetchMetricsManager(metrics); + + this.client = new MockClient(time, metadata); + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap() { + { + String topic1 = "test1"; + put(topic1, 1); + String topic2 = "test2"; + put(topic2, 1); + } + }); + this.client.updateMetadata(metadataResponse); + + this.networkClientDelegate = spy(new NetworkClientDelegate(time, + config, + logContext, + client)); + this.offsetsRequestManager = spy(new OffsetsRequestManager(subscriptions, + metadata, + fetchConfig.isolationLevel, + time, + retryBackoffMs, + requestTimeoutMs, + apiVersions, + networkClientDelegate, + backgroundEventHandler, + logContext)); + + if (groupInfo.isPresent()) { + GroupInformation gi = groupInfo.get(); + CoordinatorRequestManager coordinator = spy(new CoordinatorRequestManager( + time, + logContext, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + backgroundEventHandler, + gi.groupState.groupId + )); + CommitRequestManager commit = spy(new CommitRequestManager(time, + logContext, + subscriptions, + config, + coordinator, + groupState)); + MembershipManager mm = spy( + new MembershipManagerImpl( + gi.groupState.groupId, + gi.groupState.groupInstanceId.orElse(null), + null, + logContext + ) + ); + HeartbeatRequestManager.HeartbeatRequestState state = spy(new HeartbeatRequestManager.HeartbeatRequestState(logContext, + time, + gi.heartbeatIntervalMs, + retryBackoffMs, + DEFAULT_RETRY_BACKOFF_MAX_MS, + gi.heartbeatJitterMs)); + HeartbeatRequestManager heartbeat = spy(new HeartbeatRequestManager( + logContext, + time, + config, + coordinator, + subscriptions, + mm, + state, + backgroundEventHandler)); + + this.coordinatorRequestManager = Optional.of(coordinator); + this.commitRequestManager = Optional.of(commit); + this.heartbeatRequestManager = Optional.of(heartbeat); + this.heartbeatRequestState = Optional.of(state); + this.membershipManager = Optional.of(mm); + } else { + this.coordinatorRequestManager = Optional.empty(); + this.commitRequestManager = Optional.empty(); + this.heartbeatRequestManager = Optional.empty(); + this.heartbeatRequestState = Optional.empty(); + this.membershipManager = Optional.empty(); + } + + this.fetchBuffer = new FetchBuffer(logContext); + this.fetchRequestManager = spy(new FetchRequestManager(logContext, + time, + metadata, + subscriptions, + fetchConfig, + fetchBuffer, + metricsManager, + networkClientDelegate)); + this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, + config)); + this.requestManagers = new RequestManagers(logContext, + offsetsRequestManager, + topicMetadataRequestManager, + fetchRequestManager, + coordinatorRequestManager, + commitRequestManager, + heartbeatRequestManager); + this.applicationEventProcessor = spy(new ApplicationEventProcessor( + logContext, + applicationEventQueue, + requestManagers, + metadata) + ); + this.backgroundEventProcessor = spy(new BackgroundEventProcessor(logContext, backgroundEventQueue)); + } + + @Override + public void close() { + closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); + closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); + closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName()); + } + + public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { + + final ConsumerNetworkThread consumerNetworkThread; + + public ConsumerNetworkThreadTestBuilder() { + this(createDefaultGroupInformation()); + } + + public ConsumerNetworkThreadTestBuilder(Optional groupInfo) { + super(groupInfo); + this.consumerNetworkThread = new ConsumerNetworkThread( + logContext, + time, + () -> applicationEventProcessor, + () -> networkClientDelegate, + () -> requestManagers + ); + } + + @Override + public void close() { + closeQuietly(consumerNetworkThread, ConsumerNetworkThread.class.getSimpleName()); + } + } + + public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { + + public final ApplicationEventHandler applicationEventHandler; + + public ApplicationEventHandlerTestBuilder() { + this(createDefaultGroupInformation()); + } + + public ApplicationEventHandlerTestBuilder(Optional groupInfo) { + super(groupInfo); + this.applicationEventHandler = spy(new ApplicationEventHandler( + logContext, + time, + applicationEventQueue, + () -> applicationEventProcessor, + () -> networkClientDelegate, + () -> requestManagers)); + } + + @Override + public void close() { + closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); + } + } + + public static class PrototypeAsyncConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { + + final PrototypeAsyncConsumer consumer; + + public PrototypeAsyncConsumerTestBuilder(Optional groupInfo) { + super(groupInfo); + String clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); + List assignors = ConsumerPartitionAssignor.getAssignorInstances( + config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) + ); + Deserializers deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer()); + FetchCollector fetchCollector = new FetchCollector<>(logContext, + metadata, + subscriptions, + fetchConfig, + deserializers, + metricsManager, + time); + this.consumer = spy(new PrototypeAsyncConsumer<>( + logContext, + clientId, + deserializers, + new FetchBuffer(logContext), + fetchCollector, + new ConsumerInterceptors<>(Collections.emptyList()), + time, + applicationEventHandler, + backgroundEventQueue, + metrics, + subscriptions, + metadata, + retryBackoffMs, + 60000, + assignors, + groupInfo.map(groupInformation -> groupInformation.groupState.groupId).orElse(null))); + } + + @Override + public void close() { + consumer.close(); + } + } + + public static class GroupInformation { + + final GroupState groupState; + final int heartbeatIntervalMs; + final double heartbeatJitterMs; + + public GroupInformation(GroupState groupState) { + this(groupState, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_HEARTBEAT_JITTER_MS); + } + + public GroupInformation(GroupState groupState, int heartbeatIntervalMs, double heartbeatJitterMs) { + this.groupState = groupState; + this.heartbeatIntervalMs = heartbeatIntervalMs; + this.heartbeatJitterMs = heartbeatJitterMs; + } + } + + static Optional createDefaultGroupInformation() { + return Optional.of(new GroupInformation(new GroupState(DEFAULT_GROUP_ID, Optional.empty()))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index 4018f48fba8..23ee97daf45 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.TimeoutException; @@ -46,14 +48,14 @@ public class CoordinatorRequestManagerTest { private static final int RETRY_BACKOFF_MS = 500; private static final String GROUP_ID = "group-1"; private MockTime time; - private ErrorEventHandler errorEventHandler; + private BackgroundEventHandler backgroundEventHandler; private Node node; @BeforeEach public void setup() { this.time = new MockTime(0); this.node = new Node(1, "localhost", 9092); - this.errorEventHandler = mock(ErrorEventHandler.class); + this.backgroundEventHandler = mock(BackgroundEventHandler.class); } @Test @@ -97,7 +99,7 @@ public class CoordinatorRequestManagerTest { public void testBackoffAfterRetriableFailure() { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS); - verifyNoInteractions(errorEventHandler); + verifyNoInteractions(backgroundEventHandler); time.sleep(RETRY_BACKOFF_MS - 1); assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests); @@ -111,10 +113,15 @@ public class CoordinatorRequestManagerTest { CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID); expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED); - verify(errorEventHandler).handle(argThat(exception -> { - if (!(exception instanceof GroupAuthorizationException)) { + verify(backgroundEventHandler).add(argThat(backgroundEvent -> { + if (!(backgroundEvent instanceof ErrorBackgroundEvent)) return false; - } + + RuntimeException exception = ((ErrorBackgroundEvent) backgroundEvent).error(); + + if (!(exception instanceof GroupAuthorizationException)) + return false; + GroupAuthorizationException groupAuthException = (GroupAuthorizationException) exception; return groupAuthException.groupId().equals(GROUP_ID); })); @@ -185,7 +192,7 @@ public class CoordinatorRequestManagerTest { new LogContext(), RETRY_BACKOFF_MS, RETRY_BACKOFF_MS, - this.errorEventHandler, + this.backgroundEventHandler, groupId ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThreadTest.java deleted file mode 100644 index a7d8ac0a61a..00000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThreadTest.java +++ /dev/null @@ -1,436 +0,0 @@ -/* - * 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.GroupRebalanceConfig; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.LogTruncationException; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; -import org.apache.kafka.clients.consumer.internals.events.NoopApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TopicAuthorizationException; -import org.apache.kafka.common.message.FindCoordinatorRequestData; -import org.apache.kafka.common.requests.FindCoordinatorRequest; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mockito; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@SuppressWarnings("ClassDataAbstractionCoupling") -public class DefaultBackgroundThreadTest { - private static final long RETRY_BACKOFF_MS = 100; - private final Properties properties = new Properties(); - private MockTime time; - private ConsumerMetadata metadata; - private NetworkClientDelegate networkClient; - private BlockingQueue backgroundEventsQueue; - private BlockingQueue applicationEventsQueue; - private ApplicationEventProcessor applicationEventProcessor; - private CoordinatorRequestManager coordinatorManager; - private OffsetsRequestManager offsetsRequestManager; - private ErrorEventHandler errorEventHandler; - private final int requestTimeoutMs = 500; - private GroupState groupState; - private CommitRequestManager commitManager; - private TopicMetadataRequestManager topicMetadataRequestManager; - private HeartbeatRequestManager heartbeatRequestManager; - - @BeforeEach - @SuppressWarnings("unchecked") - public void setup() { - this.time = new MockTime(0); - this.metadata = mock(ConsumerMetadata.class); - this.networkClient = mock(NetworkClientDelegate.class); - this.applicationEventsQueue = (BlockingQueue) mock(BlockingQueue.class); - this.backgroundEventsQueue = (BlockingQueue) mock(BlockingQueue.class); - this.applicationEventProcessor = mock(ApplicationEventProcessor.class); - this.coordinatorManager = mock(CoordinatorRequestManager.class); - this.offsetsRequestManager = mock(OffsetsRequestManager.class); - this.heartbeatRequestManager = mock(HeartbeatRequestManager.class); - this.errorEventHandler = mock(ErrorEventHandler.class); - GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig( - 100, - 100, - 100, - "group_id", - Optional.empty(), - 100, - 1000, - true); - this.groupState = new GroupState(rebalanceConfig); - this.commitManager = mock(CommitRequestManager.class); - this.topicMetadataRequestManager = mock(TopicMetadataRequestManager.class); - } - - @Test - public void testStartupAndTearDown() throws InterruptedException { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - backgroundThread.start(); - TestUtils.waitForCondition(backgroundThread::isRunning, "Failed awaiting for the background thread to be running"); - backgroundThread.close(); - assertFalse(backgroundThread.isRunning()); - } - - @Test - public void testApplicationEvent() { - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - ApplicationEvent e = new NoopApplicationEvent("noop event"); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(applicationEventProcessor, times(1)).process(e); - backgroundThread.close(); - } - - @Test - public void testMetadataUpdateEvent() { - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - this.applicationEventProcessor = new ApplicationEventProcessor( - this.backgroundEventsQueue, - mockRequestManagers(), - metadata); - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - ApplicationEvent e = new NewTopicsMetadataUpdateRequestEvent(); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(metadata).requestUpdateForNewTopics(); - backgroundThread.close(); - } - - @Test - public void testCommitEvent() { - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - ApplicationEvent e = new CommitApplicationEvent(new HashMap<>()); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(CommitApplicationEvent.class)); - backgroundThread.close(); - } - - - @Test - public void testListOffsetsEventIsProcessed() { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); - assertTrue(applicationEventsQueue.isEmpty()); - backgroundThread.close(); - } - - @Test - public void testResetPositionsEventIsProcessed() { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); - assertTrue(applicationEventsQueue.isEmpty()); - backgroundThread.close(); - } - - @Test - public void testResetPositionsProcessFailure() { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - applicationEventProcessor = spy(new ApplicationEventProcessor( - this.backgroundEventsQueue, - mockRequestManagers(), - metadata)); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - - TopicAuthorizationException authException = new TopicAuthorizationException("Topic authorization failed"); - doThrow(authException).when(offsetsRequestManager).resetPositionsIfNeeded(); - - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); - this.applicationEventsQueue.add(event); - assertThrows(TopicAuthorizationException.class, backgroundThread::runOnce); - - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); - backgroundThread.close(); - } - - @Test - public void testValidatePositionsEventIsProcessed() { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); - this.applicationEventsQueue.add(e); - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); - assertTrue(applicationEventsQueue.isEmpty()); - backgroundThread.close(); - } - - @Test - public void testValidatePositionsProcessFailure() { - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - applicationEventProcessor = spy(new ApplicationEventProcessor( - this.backgroundEventsQueue, - mockRequestManagers(), - metadata)); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - - LogTruncationException logTruncationException = new LogTruncationException(Collections.emptyMap(), Collections.emptyMap()); - doThrow(logTruncationException).when(offsetsRequestManager).validatePositionsIfNeeded(); - - ValidatePositionsApplicationEvent event = new ValidatePositionsApplicationEvent(); - this.applicationEventsQueue.add(event); - assertThrows(LogTruncationException.class, backgroundThread::runOnce); - - verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); - backgroundThread.close(); - } - - @Test - public void testAssignmentChangeEvent() { - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - this.backgroundEventsQueue = new LinkedBlockingQueue<>(); - this.applicationEventProcessor = spy(new ApplicationEventProcessor( - this.backgroundEventsQueue, - mockRequestManagers(), - metadata)); - - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - HashMap offset = mockTopicPartitionOffset(); - - final long currentTimeMs = time.milliseconds(); - ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs); - this.applicationEventsQueue.add(e); - - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class)); - verify(networkClient, times(1)).poll(anyLong(), anyLong()); - verify(commitManager, times(1)).updateAutoCommitTimer(currentTimeMs); - verify(commitManager, times(1)).maybeAutoCommit(offset); - - backgroundThread.close(); - } - - @Test - void testFindCoordinator() { - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - backgroundThread.runOnce(); - Mockito.verify(coordinatorManager, times(1)).poll(anyLong()); - Mockito.verify(networkClient, times(1)).poll(anyLong(), anyLong()); - backgroundThread.close(); - } - - @Test - void testFetchTopicMetadata() { - this.applicationEventsQueue = new LinkedBlockingQueue<>(); - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult()); - when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult()); - when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults()); - this.applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic")); - backgroundThread.runOnce(); - verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); - backgroundThread.close(); - } - - @Test - void testPollResultTimer() { - DefaultBackgroundThread backgroundThread = mockBackgroundThread(); - // purposely setting a non MAX time to ensure it is returning Long.MAX_VALUE upon success - NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult( - 10, - Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs))); - assertEquals(10, backgroundThread.handlePollResult(success)); - - NetworkClientDelegate.PollResult failure = new NetworkClientDelegate.PollResult( - 10, - new ArrayList<>()); - assertEquals(10, backgroundThread.handlePollResult(failure)); - } - - private HashMap mockTopicPartitionOffset() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); - return topicPartitionOffsets; - } - - private RequestManagers mockRequestManagers() { - return new RequestManagers( - offsetsRequestManager, - topicMetadataRequestManager, - Optional.of(coordinatorManager), - Optional.of(commitManager), - Optional.of(heartbeatRequestManager)); - } - - private static NetworkClientDelegate.UnsentRequest findCoordinatorUnsentRequest( - final Time time, - final long timeout - ) { - NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest( - new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) - .setKey("foobar")), - Optional.empty()); - req.setTimer(time, timeout); - return req; - } - - private DefaultBackgroundThread mockBackgroundThread() { - properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(RETRY_BACKOFF_MS_CONFIG, RETRY_BACKOFF_MS); - - return new DefaultBackgroundThread( - this.time, - new ConsumerConfig(properties), - new LogContext(), - applicationEventsQueue, - backgroundEventsQueue, - this.errorEventHandler, - applicationEventProcessor, - this.metadata, - this.networkClient, - this.groupState, - this.coordinatorManager, - this.commitManager, - this.offsetsRequestManager, - this.topicMetadataRequestManager, - this.heartbeatRequestManager); - } - - private NetworkClientDelegate.PollResult mockPollCoordinatorResult() { - return new NetworkClientDelegate.PollResult( - RETRY_BACKOFF_MS, - Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs))); - } - - private NetworkClientDelegate.PollResult mockPollCommitResult() { - return new NetworkClientDelegate.PollResult( - RETRY_BACKOFF_MS, - Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs))); - } - - private NetworkClientDelegate.PollResult emptyPollResults() { - return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList()); - } -} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandlerTest.java deleted file mode 100644 index 7e4de791da6..00000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandlerTest.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.GroupRebalanceConfig; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.NoopApplicationEvent; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.Optional; -import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -public class DefaultEventHandlerTest { - private int sessionTimeoutMs = 1000; - private int rebalanceTimeoutMs = 1000; - private int heartbeatIntervalMs = 1000; - private String groupId = "g-1"; - private Optional groupInstanceId = Optional.of("g-1"); - private long retryBackoffMs = 1000; - private final Properties properties = new Properties(); - private GroupRebalanceConfig rebalanceConfig; - - @BeforeEach - public void setup() { - properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(RETRY_BACKOFF_MS_CONFIG, "100"); - - this.rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs, - rebalanceTimeoutMs, - heartbeatIntervalMs, - groupId, - groupInstanceId, - retryBackoffMs, - retryBackoffMs, - true); - } - - @Test - public void testBasicHandlerOps() { - final DefaultBackgroundThread bt = mock(DefaultBackgroundThread.class); - final BlockingQueue aq = new LinkedBlockingQueue<>(); - final BlockingQueue bq = new LinkedBlockingQueue<>(); - final DefaultEventHandler handler = new DefaultEventHandler(bt, aq, bq); - assertTrue(handler.isEmpty()); - assertFalse(handler.poll().isPresent()); - handler.add(new NoopApplicationEvent("test")); - assertEquals(1, aq.size()); - handler.close(); - verify(bt, times(1)).close(); - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java index 1f157b84189..0ca4a18a48a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java @@ -51,7 +51,6 @@ import java.util.Properties; import java.util.Set; import java.util.stream.Stream; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchConfig; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState; @@ -79,10 +78,11 @@ public class FetchCollectorTest { private LogContext logContext; private SubscriptionState subscriptions; - private FetchConfig fetchConfig; + private FetchConfig fetchConfig; private FetchMetricsManager metricsManager; private ConsumerMetadata metadata; private FetchBuffer fetchBuffer; + private Deserializers deserializers; private FetchCollector fetchCollector; private CompletedFetchBuilder completedFetchBuilder; @@ -193,6 +193,7 @@ public class FetchCollectorTest { metadata, subscriptions, fetchConfig, + deserializers, metricsManager, time) { @@ -427,10 +428,10 @@ public class FetchCollectorTest { ConsumerConfig config = new ConsumerConfig(p); - Deserializers deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer()); + deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer()); subscriptions = createSubscriptionState(config, logContext); - fetchConfig = createFetchConfig(config, deserializers); + fetchConfig = new FetchConfig(config); Metrics metrics = createMetrics(config, time); metricsManager = createFetchMetricsManager(metrics); @@ -448,6 +449,7 @@ public class FetchCollectorTest { metadata, subscriptions, fetchConfig, + deserializers, metricsManager, time); fetchBuffer = new FetchBuffer(logContext); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchConfigTest.java index 1f89822fdb2..f38a6294624 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchConfigTest.java @@ -19,14 +19,11 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.junit.jupiter.api.Test; import java.util.Properties; -import static org.junit.jupiter.api.Assertions.assertThrows; - public class FetchConfigTest { /** @@ -35,60 +32,27 @@ public class FetchConfigTest { */ @Test public void testBasicFromConsumerConfig() { - try (StringDeserializer keyDeserializer = new StringDeserializer(); StringDeserializer valueDeserializer = new StringDeserializer()) { - newFetchConfigFromConsumerConfig(keyDeserializer, valueDeserializer); - newFetchConfigFromValues(keyDeserializer, valueDeserializer); - } - } - - /** - * Verify an exception is thrown if the key {@link Deserializer deserializer} provided to the - * {@link FetchConfig} constructors is {@code null}. - */ - @Test - public void testPreventNullKeyDeserializer() { - try (StringDeserializer valueDeserializer = new StringDeserializer()) { - assertThrows(NullPointerException.class, () -> newFetchConfigFromConsumerConfig(null, valueDeserializer)); - assertThrows(NullPointerException.class, () -> newFetchConfigFromValues(null, valueDeserializer)); - } + newFetchConfigFromConsumerConfig(); + newFetchConfigFromValues(); } - /** - * Verify an exception is thrown if the value {@link Deserializer deserializer} provided to the - * {@link FetchConfig} constructors is {@code null}. - */ - @Test - @SuppressWarnings("resources") - public void testPreventNullValueDeserializer() { - try (StringDeserializer keyDeserializer = new StringDeserializer()) { - assertThrows(NullPointerException.class, () -> newFetchConfigFromConsumerConfig(keyDeserializer, null)); - assertThrows(NullPointerException.class, () -> newFetchConfigFromValues(keyDeserializer, null)); - } - } - - private void newFetchConfigFromConsumerConfig(Deserializer keyDeserializer, - Deserializer valueDeserializer) { + private void newFetchConfigFromConsumerConfig() { Properties p = new Properties(); p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); ConsumerConfig config = new ConsumerConfig(p); - new FetchConfig<>( - config, - new Deserializers<>(keyDeserializer, valueDeserializer), - IsolationLevel.READ_UNCOMMITTED); + new FetchConfig(config); } - private void newFetchConfigFromValues(Deserializer keyDeserializer, - Deserializer valueDeserializer) { - new FetchConfig<>(ConsumerConfig.DEFAULT_FETCH_MIN_BYTES, + private void newFetchConfigFromValues() { + new FetchConfig(ConsumerConfig.DEFAULT_FETCH_MIN_BYTES, ConsumerConfig.DEFAULT_FETCH_MAX_BYTES, ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS, ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES, ConsumerConfig.DEFAULT_MAX_POLL_RECORDS, true, ConsumerConfig.DEFAULT_CLIENT_RACK, - new Deserializers<>(keyDeserializer, valueDeserializer), IsolationLevel.READ_UNCOMMITTED); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java new file mode 100644 index 00000000000..c4270c3f4ca --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -0,0 +1,3512 @@ +/* + * 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.ClientRequest; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.NodeApiVersions; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.IsolationLevel; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.MetricNameTemplate; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.internals.ClusterResourceListeners; +import org.apache.kafka.common.message.ApiMessageType; +import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset; +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.OffsetForLeaderTopicResult; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.network.NetworkReceive; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.DefaultRecordBatch; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.LegacyRecord; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.requests.ApiVersionsResponse; +import org.apache.kafka.common.requests.FetchMetadata; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.FetchRequest.PartitionData; +import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; +import org.apache.kafka.common.requests.RequestTestUtils; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.BytesDeserializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.ByteBufferOutputStream; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.DelayedReceive; +import org.apache.kafka.test.MockSelector; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.ArgumentCaptor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.test.TestUtils.assertOptional; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class FetchRequestManagerTest { + + private static final double EPSILON = 0.0001; + + private ConsumerRebalanceListener listener = new NoOpConsumerRebalanceListener(); + private String topicName = "test"; + private String groupId = "test-group"; + private Uuid topicId = Uuid.randomUuid(); + private Map topicIds = new HashMap() { + { + put(topicName, topicId); + } + }; + private Map topicNames = singletonMap(topicId, topicName); + private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; + private TopicPartition tp0 = new TopicPartition(topicName, 0); + private TopicPartition tp1 = new TopicPartition(topicName, 1); + private TopicPartition tp2 = new TopicPartition(topicName, 2); + private TopicPartition tp3 = new TopicPartition(topicName, 3); + private TopicIdPartition tidp0 = new TopicIdPartition(topicId, tp0); + private TopicIdPartition tidp1 = new TopicIdPartition(topicId, tp1); + private TopicIdPartition tidp2 = new TopicIdPartition(topicId, tp2); + private TopicIdPartition tidp3 = new TopicIdPartition(topicId, tp3); + private int validLeaderEpoch = 0; + private MetadataResponse initialUpdateResponse = + RequestTestUtils.metadataUpdateWithIds(1, singletonMap(topicName, 4), topicIds); + + private int minBytes = 1; + private int maxBytes = Integer.MAX_VALUE; + private int maxWaitMs = 0; + private int fetchSize = 1000; + private long retryBackoffMs = 100; + private long requestTimeoutMs = 30000; + private MockTime time = new MockTime(1); + private SubscriptionState subscriptions; + private ConsumerMetadata metadata; + private FetchMetricsRegistry metricsRegistry; + private FetchMetricsManager metricsManager; + private MockClient client; + private Metrics metrics; + private ApiVersions apiVersions = new ApiVersions(); + private TestableFetchRequestManager fetcher; + private TestableNetworkClientDelegate networkClientDelegate; + private OffsetFetcher offsetFetcher; + + private MemoryRecords records; + private MemoryRecords nextRecords; + private MemoryRecords emptyRecords; + private MemoryRecords partialRecords; + + @BeforeEach + public void setup() { + records = buildRecords(1L, 3, 1); + nextRecords = buildRecords(4L, 2, 4); + emptyRecords = buildRecords(0L, 0, 0); + partialRecords = buildRecords(4L, 1, 0); + partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); + } + + private void assignFromUser(Set partitions) { + subscriptions.assignFromUser(partitions); + client.updateMetadata(initialUpdateResponse); + + // A dummy metadata update to ensure valid leader epoch. + metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWithIds("dummy", 1, + Collections.emptyMap(), singletonMap(topicName, 4), + tp -> validLeaderEpoch, topicIds), false, 0L); + } + + private void assignFromUser(TopicPartition partition) { + subscriptions.assignFromUser(singleton(partition)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singletonMap(partition.topic(), 1), Collections.emptyMap())); + + // A dummy metadata update to ensure valid leader epoch. + metadata.update(9, RequestTestUtils.metadataUpdateWithIds("dummy", 1, + Collections.emptyMap(), singletonMap(partition.topic(), 1), + tp -> validLeaderEpoch, topicIds), false, 0L); + } + + @AfterEach + public void teardown() throws Exception { + if (metrics != null) + metrics.close(); + if (fetcher != null) + fetcher.close(); + } + + private int sendFetches() { + offsetFetcher.validatePositionsOnMetadataChange(); + return fetcher.sendFetches(); + } + + @Test + public void testFetchNormal() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + + List> records = partitionRecords.get(tp0); + assertEquals(3, records.size()); + assertEquals(4L, subscriptions.position(tp0).offset); // this is the next fetching position + long offset = 1; + for (ConsumerRecord record : records) { + assertEquals(offset, record.offset()); + offset += 1; + } + } + + @Test + public void testInflightFetchOnPendingPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + subscriptions.markPendingRevocation(singleton(tp0)); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertNull(fetchRecords().get(tp0)); + } + + @Test + public void testCloseShouldBeIdempotent() { + buildFetcher(); + + fetcher.close(); + fetcher.close(); + fetcher.close(); + + verify(fetcher, times(1)).closeInternal(any(Timer.class)); + } + + @Test + public void testFetcherCloseClosesFetchSessionsInBroker() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + final FetchResponse fetchResponse = fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0); + client.prepareResponse(fetchResponse); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + assertEquals(0, networkClientDelegate.pendingRequestCount()); + + final ArgumentCaptor argument = ArgumentCaptor.forClass(NetworkClientDelegate.UnsentRequest.class); + + Timer timer = time.timer(Duration.ofSeconds(10)); + // NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling + // the close() method with a Timer will NOT send out the close session requests on close. The network + // I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here. + ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer); + + // validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the + // normal fetch earlier and another for the finish fetch here. + verify(networkClientDelegate, times(2)).doSend(argument.capture(), any(Long.class)); + NetworkClientDelegate.UnsentRequest lastUnsentRequest = argument.getValue(); + FetchRequest.Builder builder = (FetchRequest.Builder) lastUnsentRequest.requestBuilder(); + // session Id is the same + assertEquals(fetchResponse.sessionId(), builder.metadata().sessionId()); + // contains final epoch + assertEquals(FetchMetadata.FINAL_EPOCH, builder.metadata().epoch()); // final epoch indicates we want to close the session + assertTrue(builder.fetchData().isEmpty()); // partition data should be empty + } + + @Test + public void testFetchingPendingPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + assertEquals(4L, subscriptions.position(tp0).offset); // this is the next fetching position + + // mark partition unfetchable + subscriptions.markPendingRevocation(singleton(tp0)); + assertEquals(0, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchRecords(); + assertEquals(4L, subscriptions.position(tp0).offset); + } + + @Test + public void testFetchWithNoTopicId() { + // Should work and default to using old request type. + buildFetcher(); + + TopicIdPartition noId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("noId", 0)); + assignFromUser(noId.topicPartition()); + subscriptions.seek(noId.topicPartition(), 0); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Fetch should use request version 12 + client.prepareResponse( + fetchRequestMatcher((short) 12, noId, 0, Optional.of(validLeaderEpoch)), + fullFetchResponse(noId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(noId.topicPartition())); + + List> records = partitionRecords.get(noId.topicPartition()); + assertEquals(3, records.size()); + assertEquals(4L, subscriptions.position(noId.topicPartition()).offset); // this is the next fetching position + long offset = 1; + for (ConsumerRecord record : records) { + assertEquals(offset, record.offset()); + offset += 1; + } + } + + @Test + public void testFetchWithTopicId() { + buildFetcher(); + + TopicIdPartition tp = new TopicIdPartition(topicId, new TopicPartition(topicName, 0)); + assignFromUser(singleton(tp.topicPartition())); + subscriptions.seek(tp.topicPartition(), 0); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Fetch should use latest version + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), tp, 0, Optional.of(validLeaderEpoch)), + fullFetchResponse(tp, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp.topicPartition())); + + List> records = partitionRecords.get(tp.topicPartition()); + assertEquals(3, records.size()); + assertEquals(4L, subscriptions.position(tp.topicPartition()).offset); // this is the next fetching position + long offset = 1; + for (ConsumerRecord record : records) { + assertEquals(offset, record.offset()); + offset += 1; + } + } + + @Test + public void testFetchForgetTopicIdWhenUnassigned() { + buildFetcher(); + + TopicIdPartition foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0)); + + // Assign foo and bar. + subscriptions.assignFromUser(singleton(foo.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(foo), tp -> validLeaderEpoch)); + subscriptions.seek(foo.topicPartition(), 0); + + assertEquals(1, sendFetches()); + + // Fetch should use latest version. + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), + singletonMap(foo, new PartitionData( + foo.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + emptyList() + ), + fullFetchResponse(1, foo, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Assign bar and un-assign foo. + subscriptions.assignFromUser(singleton(bar.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(bar), tp -> validLeaderEpoch)); + subscriptions.seek(bar.topicPartition(), 0); + + // Fetch should use latest version. + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), + singletonMap(bar, new PartitionData( + bar.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + singletonList(foo) + ), + fullFetchResponse(1, bar, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + } + + @Test + public void testFetchForgetTopicIdWhenReplaced() { + buildFetcher(); + + TopicIdPartition fooWithOldTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition fooWithNewTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + + // Assign foo with old topic id. + subscriptions.assignFromUser(singleton(fooWithOldTopicId.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithOldTopicId), tp -> validLeaderEpoch)); + subscriptions.seek(fooWithOldTopicId.topicPartition(), 0); + + // Fetch should use latest version. + assertEquals(1, sendFetches()); + + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), + singletonMap(fooWithOldTopicId, new PartitionData( + fooWithOldTopicId.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + emptyList() + ), + fullFetchResponse(1, fooWithOldTopicId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Replace foo with old topic id with foo with new topic id. + subscriptions.assignFromUser(singleton(fooWithNewTopicId.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithNewTopicId), tp -> validLeaderEpoch)); + subscriptions.seek(fooWithNewTopicId.topicPartition(), 0); + + // Fetch should use latest version. + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // foo with old topic id should be removed from the session. + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), + singletonMap(fooWithNewTopicId, new PartitionData( + fooWithNewTopicId.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + singletonList(fooWithOldTopicId) + ), + fullFetchResponse(1, fooWithNewTopicId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + } + + @Test + public void testFetchTopicIdUpgradeDowngrade() { + buildFetcher(); + + TopicIdPartition fooWithoutId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("foo", 0)); + + // Assign foo without a topic id. + subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithoutId), tp -> validLeaderEpoch)); + subscriptions.seek(fooWithoutId.topicPartition(), 0); + + // Fetch should use version 12. + assertEquals(1, sendFetches()); + + client.prepareResponse( + fetchRequestMatcher((short) 12, + singletonMap(fooWithoutId, new PartitionData( + fooWithoutId.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + emptyList() + ), + fullFetchResponse(1, fooWithoutId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Upgrade. + TopicIdPartition fooWithId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + subscriptions.assignFromUser(singleton(fooWithId.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithId), tp -> validLeaderEpoch)); + subscriptions.seek(fooWithId.topicPartition(), 0); + + // Fetch should use latest version. + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // foo with old topic id should be removed from the session. + client.prepareResponse( + fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), + singletonMap(fooWithId, new PartitionData( + fooWithId.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + emptyList() + ), + fullFetchResponse(1, fooWithId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Downgrade. + subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition())); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithoutId), tp -> validLeaderEpoch)); + subscriptions.seek(fooWithoutId.topicPartition(), 0); + + // Fetch should use version 12. + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // foo with old topic id should be removed from the session. + client.prepareResponse( + fetchRequestMatcher((short) 12, + singletonMap(fooWithoutId, new PartitionData( + fooWithoutId.topicId(), + 0, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + Optional.of(validLeaderEpoch)) + ), + emptyList() + ), + fullFetchResponse(1, fooWithoutId, records, Errors.NONE, 100L, 0) + ); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + } + + private MockClient.RequestMatcher fetchRequestMatcher( + short expectedVersion, + TopicIdPartition tp, + long expectedFetchOffset, + Optional expectedCurrentLeaderEpoch + ) { + return fetchRequestMatcher( + expectedVersion, + singletonMap(tp, new PartitionData( + tp.topicId(), + expectedFetchOffset, + FetchRequest.INVALID_LOG_START_OFFSET, + fetchSize, + expectedCurrentLeaderEpoch + )), + emptyList() + ); + } + + private MockClient.RequestMatcher fetchRequestMatcher( + short expectedVersion, + Map fetch, + List forgotten + ) { + return body -> { + if (body instanceof FetchRequest) { + FetchRequest fetchRequest = (FetchRequest) body; + assertEquals(expectedVersion, fetchRequest.version()); + assertEquals(fetch, fetchRequest.fetchData(topicNames(new ArrayList<>(fetch.keySet())))); + assertEquals(forgotten, fetchRequest.forgottenTopics(topicNames(forgotten))); + return true; + } else { + fail("Should have seen FetchRequest"); + return false; + } + }; + } + + private Map topicNames(List partitions) { + Map topicNames = new HashMap<>(); + partitions.forEach(partition -> topicNames.putIfAbsent(partition.topicId(), partition.topic())); + return topicNames; + } + + @Test + public void testMissingLeaderEpochInRecords() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + ByteBuffer buffer = ByteBuffer.allocate(1024); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, + CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), + RecordBatch.NO_PARTITION_LEADER_EPOCH); + builder.append(0L, "key".getBytes(), "1".getBytes()); + builder.append(0L, "key".getBytes(), "2".getBytes()); + MemoryRecords records = builder.build(); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + assertEquals(2, partitionRecords.get(tp0).size()); + + for (ConsumerRecord record : partitionRecords.get(tp0)) { + assertEquals(Optional.empty(), record.leaderEpoch()); + } + } + + @Test + public void testLeaderEpochInConsumerRecord() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + Integer partitionLeaderEpoch = 1; + + ByteBuffer buffer = ByteBuffer.allocate(1024); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), + partitionLeaderEpoch); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.close(); + + partitionLeaderEpoch += 7; + + builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.close(); + + partitionLeaderEpoch += 5; + builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes()); + builder.close(); + + buffer.flip(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + assertEquals(6, partitionRecords.get(tp0).size()); + + for (ConsumerRecord record : partitionRecords.get(tp0)) { + int expectedLeaderEpoch = Integer.parseInt(Utils.utf8(record.value())); + assertEquals(Optional.of(expectedLeaderEpoch), record.leaderEpoch()); + } + } + + @Test + public void testClearBufferedDataForTopicPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + Set newAssignedTopicPartitions = new HashSet<>(); + newAssignedTopicPartitions.add(tp1); + + fetcher.clearBufferedDataForUnassignedPartitions(newAssignedTopicPartitions); + assertFalse(fetcher.hasCompletedFetches()); + } + + @Test + public void testFetchSkipsBlackedOutNodes() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + Node node = initialUpdateResponse.brokers().iterator().next(); + + client.backoff(node, 500); + assertEquals(0, sendFetches()); + + time.sleep(500); + assertEquals(1, sendFetches()); + } + + @Test + public void testFetcherIgnoresControlRecords() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + long producerId = 1; + short producerEpoch = 0; + int baseSequence = 0; + int partitionLeaderEpoch = 0; + + ByteBuffer buffer = ByteBuffer.allocate(1024); + MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, CompressionType.NONE, 0L, producerId, + producerEpoch, baseSequence); + builder.append(0L, "key".getBytes(), null); + builder.close(); + + MemoryRecords.writeEndTransactionalMarker(buffer, 1L, time.milliseconds(), partitionLeaderEpoch, producerId, producerEpoch, + new EndTransactionMarker(ControlRecordType.ABORT, 0)); + + buffer.flip(); + + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + + List> records = partitionRecords.get(tp0); + assertEquals(1, records.size()); + assertEquals(2L, subscriptions.position(tp0).offset); + + ConsumerRecord record = records.get(0); + assertArrayEquals("key".getBytes(), record.key()); + } + + @Test + public void testFetchError() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertFalse(partitionRecords.containsKey(tp0)); + } + + private MockClient.RequestMatcher matchesOffset(final TopicIdPartition tp, final long offset) { + return body -> { + FetchRequest fetch = (FetchRequest) body; + Map fetchData = fetch.fetchData(topicNames); + return fetchData.containsKey(tp) && + fetchData.get(tp).fetchOffset == offset; + }; + } + + @Test + public void testFetchedRecordsRaisesOnSerializationErrors() { + // raise an exception from somewhere in the middle of the fetch response + // so that we can verify that our position does not advance after raising + ByteArrayDeserializer deserializer = new ByteArrayDeserializer() { + int i = 0; + @Override + public byte[] deserialize(String topic, byte[] data) { + if (i++ % 2 == 1) { + // Should be blocked on the value deserialization of the first record. + assertEquals("value-1", new String(data, StandardCharsets.UTF_8)); + throw new SerializationException(); + } + return data; + } + }; + + buildFetcher(deserializer, deserializer); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 1); + + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + + for (int i = 0; i < 2; i++) { + // The fetcher should throw a Deserialization error + assertThrows(SerializationException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(1, subscriptions.position(tp0).offset); + } + } + + @Test + public void testParseCorruptedRecord() throws Exception { + buildFetcher(); + assignFromUser(singleton(tp0)); + + ByteBuffer buffer = ByteBuffer.allocate(1024); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer)); + + byte magic = RecordBatch.MAGIC_VALUE_V1; + byte[] key = "foo".getBytes(); + byte[] value = "baz".getBytes(); + long offset = 0; + long timestamp = 500L; + + int size = LegacyRecord.recordSize(magic, key.length, value.length); + byte attributes = LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME); + long crc = LegacyRecord.computeChecksum(magic, attributes, timestamp, key, value); + + // write one valid record + out.writeLong(offset); + out.writeInt(size); + LegacyRecord.write(out, magic, crc, LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value); + + // and one invalid record (note the crc) + out.writeLong(offset + 1); + out.writeInt(size); + LegacyRecord.write(out, magic, crc + 1, LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value); + + // write one valid record + out.writeLong(offset + 2); + out.writeInt(size); + LegacyRecord.write(out, magic, crc, LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value); + + // Write a record whose size field is invalid. + out.writeLong(offset + 3); + out.writeInt(1); + + // write one valid record + out.writeLong(offset + 4); + out.writeInt(size); + LegacyRecord.write(out, magic, crc, LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value); + + buffer.flip(); + + subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), metadata.currentLeader(tp0))); + + // normal fetch + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + // the first fetchRecords() should return the first valid message + assertEquals(1, fetchRecords().get(tp0).size()); + assertEquals(1, subscriptions.position(tp0).offset); + + ensureBlockOnRecord(1L); + seekAndConsumeRecord(buffer, 2L); + ensureBlockOnRecord(3L); + try { + // For a record that cannot be retrieved from the iterator, we cannot seek over it within the batch. + seekAndConsumeRecord(buffer, 4L); + fail("Should have thrown exception when fail to retrieve a record from iterator."); + } catch (KafkaException ke) { + // let it go + } + ensureBlockOnRecord(4L); + } + + private void ensureBlockOnRecord(long blockedOffset) { + for (int i = 0; i < 2; i++) { + // the fetchRecords() should always throw exception due to the invalid message at the starting offset. + assertThrows(KafkaException.class, this::fetchRecords); + assertEquals(blockedOffset, subscriptions.position(tp0).offset); + } + } + + private void seekAndConsumeRecord(ByteBuffer responseBuffer, long toOffset) { + // Seek to skip the bad record and fetch again. + subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(toOffset, Optional.empty(), metadata.currentLeader(tp0))); + // Should not throw exception after the seek. + collectFetch(); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(responseBuffer), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + Map>> recordsByPartition = fetchRecords(); + List> records = recordsByPartition.get(tp0); + assertEquals(1, records.size()); + assertEquals(toOffset, records.get(0).offset()); + assertEquals(toOffset + 1, subscriptions.position(tp0).offset); + } + + @Test + public void testInvalidDefaultRecordBatch() { + buildFetcher(); + + ByteBuffer buffer = ByteBuffer.allocate(1024); + ByteBufferOutputStream out = new ByteBufferOutputStream(buffer); + + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out, + DefaultRecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + 0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024); + builder.append(10L, "key".getBytes(), "value".getBytes()); + builder.close(); + buffer.flip(); + + // Garble the CRC + buffer.position(17); + buffer.put("beef".getBytes()); + buffer.position(0); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + for (int i = 0; i < 2; i++) { + // the fetchRecords() should always throw exception due to the bad batch. + assertThrows(KafkaException.class, this::collectFetch); + assertEquals(0, subscriptions.position(tp0).offset); + } + } + + @Test + public void testParseInvalidRecordBatch() { + buildFetcher(); + MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, + CompressionType.NONE, TimestampType.CREATE_TIME, + new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), + new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), + new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); + ByteBuffer buffer = records.buffer(); + + // flip some bits to fail the crc + buffer.putInt(32, buffer.get(32) ^ 87238423); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + assertThrows(KafkaException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp0).offset); + } + + @Test + public void testHeaders() { + buildFetcher(); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); + builder.append(0L, "key".getBytes(), "value-1".getBytes()); + + Header[] headersArray = new Header[1]; + headersArray[0] = new RecordHeader("headerKey", "headerValue".getBytes(StandardCharsets.UTF_8)); + builder.append(0L, "key".getBytes(), "value-2".getBytes(), headersArray); + + Header[] headersArray2 = new Header[2]; + headersArray2[0] = new RecordHeader("headerKey", "headerValue".getBytes(StandardCharsets.UTF_8)); + headersArray2[1] = new RecordHeader("headerKey", "headerValue2".getBytes(StandardCharsets.UTF_8)); + builder.append(0L, "key".getBytes(), "value-3".getBytes(), headersArray2); + + MemoryRecords memoryRecords = builder.build(); + + List> records; + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 1); + + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, memoryRecords, Errors.NONE, 100L, 0)); + + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + Map>> recordsByPartition = fetchRecords(); + records = recordsByPartition.get(tp0); + + assertEquals(3, records.size()); + + Iterator> recordIterator = records.iterator(); + + ConsumerRecord record = recordIterator.next(); + assertNull(record.headers().lastHeader("headerKey")); + + record = recordIterator.next(); + assertEquals("headerValue", new String(record.headers().lastHeader("headerKey").value(), StandardCharsets.UTF_8)); + assertEquals("headerKey", record.headers().lastHeader("headerKey").key()); + + record = recordIterator.next(); + assertEquals("headerValue2", new String(record.headers().lastHeader("headerKey").value(), StandardCharsets.UTF_8)); + assertEquals("headerKey", record.headers().lastHeader("headerKey").key()); + } + + @Test + public void testFetchMaxPollRecords() { + buildFetcher(2); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 1); + + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp0, 4), fullFetchResponse(tidp0, nextRecords, Errors.NONE, 100L, 0)); + + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + Map>> recordsByPartition = fetchRecords(); + List> recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); + assertEquals(3L, subscriptions.position(tp0).offset); + assertEquals(1, recordsToTest.get(0).offset()); + assertEquals(2, recordsToTest.get(1).offset()); + + assertEquals(0, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + recordsByPartition = fetchRecords(); + recordsToTest = recordsByPartition.get(tp0); + assertEquals(1, recordsToTest.size()); + assertEquals(4L, subscriptions.position(tp0).offset); + assertEquals(3, recordsToTest.get(0).offset()); + + assertTrue(sendFetches() > 0); + networkClientDelegate.poll(time.timer(0)); + recordsByPartition = fetchRecords(); + recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); + assertEquals(6L, subscriptions.position(tp0).offset); + assertEquals(4, recordsToTest.get(0).offset()); + assertEquals(5, recordsToTest.get(1).offset()); + } + + /** + * Test the scenario where a partition with fetched but not consumed records (i.e. max.poll.records is + * less than the number of fetched records) is unassigned and a different partition is assigned. This is a + * pattern used by Streams state restoration and KAFKA-5097 would have been caught by this test. + */ + @Test + public void testFetchAfterPartitionWithFetchedRecordsIsUnassigned() { + buildFetcher(2); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 1); + + // Returns 3 records while `max.poll.records` is configured to 2 + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + Map>> recordsByPartition = fetchRecords(); + List> recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); + assertEquals(3L, subscriptions.position(tp0).offset); + assertEquals(1, recordsToTest.get(0).offset()); + assertEquals(2, recordsToTest.get(1).offset()); + + assignFromUser(singleton(tp1)); + client.prepareResponse(matchesOffset(tidp1, 4), fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); + subscriptions.seek(tp1, 4); + + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + Map>> fetchedRecords = fetchRecords(); + assertNull(fetchedRecords.get(tp0)); + recordsToTest = fetchedRecords.get(tp1); + assertEquals(2, recordsToTest.size()); + assertEquals(6L, subscriptions.position(tp1).offset); + assertEquals(4, recordsToTest.get(0).offset()); + assertEquals(5, recordsToTest.get(1).offset()); + } + + @Test + public void testFetchNonContinuousRecords() { + // if we are fetching from a compacted topic, there may be gaps in the returned records + // this test verifies the fetcher updates the current fetched/consumed positions correctly for this case + buildFetcher(); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes()); + builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes()); + builder.appendWithOffset(30L, 0L, "key".getBytes(), "value-3".getBytes()); + MemoryRecords records = builder.build(); + + List> consumerRecords; + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + Map>> recordsByPartition = fetchRecords(); + consumerRecords = recordsByPartition.get(tp0); + assertEquals(3, consumerRecords.size()); + assertEquals(31L, subscriptions.position(tp0).offset); // this is the next fetching position + + assertEquals(15L, consumerRecords.get(0).offset()); + assertEquals(20L, consumerRecords.get(1).offset()); + assertEquals(30L, consumerRecords.get(2).offset()); + } + + /** + * Test the case where the client makes a pre-v3 FetchRequest, but the server replies with only a partial + * request. This happens when a single message is larger than the per-partition limit. + */ + @Test + public void testFetchRequestWhenRecordTooLarge() { + try { + buildFetcher(); + + client.setNodeApiVersions(NodeApiVersions.create(ApiKeys.FETCH.id, (short) 2, (short) 2)); + makeFetchRequestWithIncompleteRecord(); + assertThrows(RecordTooLargeException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp0).offset); + } finally { + client.setNodeApiVersions(NodeApiVersions.create()); + } + } + + /** + * Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a + * partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior + * so that at least one message is always returned. Therefore, this case should not happen, and it indicates + * that an internal error has taken place. + */ + @Test + public void testFetchRequestInternalError() { + buildFetcher(); + makeFetchRequestWithIncompleteRecord(); + try { + collectFetch(); + fail("collectFetch should have thrown a KafkaException"); + } catch (KafkaException e) { + assertTrue(e.getMessage().startsWith("Failed to make progress reading messages")); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp0).offset); + } + } + + private void makeFetchRequestWithIncompleteRecord() { + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + MemoryRecords partialRecord = MemoryRecords.readableRecords( + ByteBuffer.wrap(new byte[]{0, 0, 0, 0, 0, 0, 0, 0})); + client.prepareResponse(fullFetchResponse(tidp0, partialRecord, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + } + + @Test + public void testUnauthorizedTopic() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.TOPIC_AUTHORIZATION_FAILED, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + try { + collectFetch(); + fail("collectFetch should have thrown a TopicAuthorizationException"); + } catch (TopicAuthorizationException e) { + assertEquals(singleton(topicName), e.unauthorizedTopics()); + } + } + + @Test + public void testFetchDuringEagerRebalance() { + buildFetcher(); + + subscriptions.subscribe(singleton(topicName), listener); + subscriptions.assignFromSubscribed(singleton(tp0)); + subscriptions.seek(tp0, 0); + + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds( + 1, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds)); + + assertEquals(1, sendFetches()); + + // Now the eager rebalance happens and fetch positions are cleared + subscriptions.assignFromSubscribed(Collections.emptyList()); + + subscriptions.assignFromSubscribed(singleton(tp0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + // The active fetch should be ignored since its position is no longer valid + assertTrue(fetchRecords().isEmpty()); + } + + @Test + public void testFetchDuringCooperativeRebalance() { + buildFetcher(); + + subscriptions.subscribe(singleton(topicName), listener); + subscriptions.assignFromSubscribed(singleton(tp0)); + subscriptions.seek(tp0, 0); + + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds( + 1, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds)); + + assertEquals(1, sendFetches()); + + // Now the cooperative rebalance happens and fetch positions are NOT cleared for unrevoked partitions + subscriptions.assignFromSubscribed(singleton(tp0)); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + Map>> fetchedRecords = fetchRecords(); + + // The active fetch should NOT be ignored since the position for tp0 is still valid + assertEquals(1, fetchedRecords.size()); + assertEquals(3, fetchedRecords.get(tp0).size()); + } + + @Test + public void testInFlightFetchOnPausedPartition() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + subscriptions.pause(tp0); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertNull(fetchRecords().get(tp0)); + } + + @Test + public void testFetchOnPausedPartition() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + subscriptions.pause(tp0); + assertFalse(sendFetches() > 0); + assertTrue(client.requests().isEmpty()); + } + + @Test + public void testFetchOnCompletedFetchesForPausedAndResumedPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + + subscriptions.pause(tp0); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + assertEmptyFetch("Should not return any records or advance position when partition is paused"); + + assertTrue(fetcher.hasCompletedFetches(), "Should still contain completed fetches"); + assertFalse(fetcher.hasAvailableFetches(), "Should not have any available (non-paused) completed fetches"); + assertEquals(0, sendFetches()); + + subscriptions.resume(tp0); + + assertTrue(fetcher.hasAvailableFetches(), "Should have available (non-paused) completed fetches"); + + networkClientDelegate.poll(time.timer(0)); + Map>> fetchedRecords = fetchRecords(); + assertEquals(1, fetchedRecords.size(), "Should return records when partition is resumed"); + assertNotNull(fetchedRecords.get(tp0)); + assertEquals(3, fetchedRecords.get(tp0).size()); + + networkClientDelegate.poll(time.timer(0)); + assertEmptyFetch("Should not return records or advance position after previously paused partitions are fetched"); + assertFalse(fetcher.hasCompletedFetches(), "Should no longer contain completed fetches"); + } + + @Test + public void testFetchOnCompletedFetchesForSomePausedPartitions() { + buildFetcher(); + + Map>> fetchedRecords; + + assignFromUser(mkSet(tp0, tp1)); + + // seek to tp0 and tp1 in two polls to generate 2 complete requests and responses + + // #1 seek, request, poll, response + subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp0))); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + // #2 seek, request, poll, response + subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); + + subscriptions.pause(tp0); + networkClientDelegate.poll(time.timer(0)); + + fetchedRecords = fetchRecords(); + assertEquals(1, fetchedRecords.size(), "Should return completed fetch for unpaused partitions"); + assertTrue(fetcher.hasCompletedFetches(), "Should still contain completed fetches"); + assertNotNull(fetchedRecords.get(tp1)); + assertNull(fetchedRecords.get(tp0)); + + assertEmptyFetch("Should not return records or advance position for remaining paused partition"); + assertTrue(fetcher.hasCompletedFetches(), "Should still contain completed fetches"); + } + + @Test + public void testFetchOnCompletedFetchesForAllPausedPartitions() { + buildFetcher(); + + assignFromUser(mkSet(tp0, tp1)); + + // seek to tp0 and tp1 in two polls to generate 2 complete requests and responses + + // #1 seek, request, poll, response + subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp0))); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + // #2 seek, request, poll, response + subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); + + subscriptions.pause(tp0); + subscriptions.pause(tp1); + + networkClientDelegate.poll(time.timer(0)); + + assertEmptyFetch("Should not return records or advance position for all paused partitions"); + assertTrue(fetcher.hasCompletedFetches(), "Should still contain completed fetches"); + assertFalse(fetcher.hasAvailableFetches(), "Should not have any available (non-paused) completed fetches"); + } + + @Test + public void testPartialFetchWithPausedPartitions() { + // this test sends creates a completed fetch with 3 records and a max poll of 2 records to assert + // that a fetch that must be returned over at least 2 polls can be cached successfully when its partition is + // paused, then returned successfully after it has been resumed again later + buildFetcher(2); + + Map>> fetchedRecords; + + assignFromUser(mkSet(tp0, tp1)); + + subscriptions.seek(tp0, 1); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + fetchedRecords = fetchRecords(); + + assertEquals(2, fetchedRecords.get(tp0).size(), "Should return 2 records from fetch with 3 records"); + assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); + + subscriptions.pause(tp0); + networkClientDelegate.poll(time.timer(0)); + + assertEmptyFetch("Should not return records or advance position for paused partitions"); + assertTrue(fetcher.hasCompletedFetches(), "Should have 1 entry in completed fetches"); + assertFalse(fetcher.hasAvailableFetches(), "Should not have any available (non-paused) completed fetches"); + + subscriptions.resume(tp0); + + networkClientDelegate.poll(time.timer(0)); + + fetchedRecords = fetchRecords(); + + assertEquals(1, fetchedRecords.get(tp0).size(), "Should return last remaining record"); + assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); + } + + @Test + public void testFetchDiscardedAfterPausedPartitionResumedAndSeekedToNewOffset() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + subscriptions.pause(tp0); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + + subscriptions.seek(tp0, 3); + subscriptions.resume(tp0); + networkClientDelegate.poll(time.timer(0)); + + assertTrue(fetcher.hasCompletedFetches(), "Should have 1 entry in completed fetches"); + Fetch fetch = collectFetch(); + assertEquals(emptyMap(), fetch.records(), "Should not return any records because we seeked to a new offset"); + assertFalse(fetch.positionAdvanced()); + assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); + } + + @Test + public void testFetchSessionIdError() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + client.prepareResponse(fetchResponseWithTopLevelError(tidp0, Errors.FETCH_SESSION_TOPIC_ID_ERROR, 0)); + networkClientDelegate.poll(time.timer(0)); + assertEmptyFetch("Should not return records or advance position on fetch error"); + assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + } + + @ParameterizedTest + @MethodSource("handleFetchResponseErrorSupplier") + public void testHandleFetchResponseError(Errors error, + long highWatermark, + boolean hasTopLevelError, + boolean shouldRequestMetadataUpdate) { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + + final FetchResponse fetchResponse; + + if (hasTopLevelError) + fetchResponse = fetchResponseWithTopLevelError(tidp0, error, 0); + else + fetchResponse = fullFetchResponse(tidp0, records, error, highWatermark, 0); + + client.prepareResponse(fetchResponse); + networkClientDelegate.poll(time.timer(0)); + + assertEmptyFetch("Should not return records or advance position on fetch error"); + + long timeToNextUpdate = metadata.timeToNextUpdate(time.milliseconds()); + + if (shouldRequestMetadataUpdate) + assertEquals(0L, timeToNextUpdate, "Should have requested metadata update"); + else + assertNotEquals(0L, timeToNextUpdate, "Should not have requested metadata update"); + } + + /** + * Supplies parameters to {@link #testHandleFetchResponseError(Errors, long, boolean, boolean)}. + */ + private static Stream handleFetchResponseErrorSupplier() { + return Stream.of( + Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER, 100L, false, true), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, 100L, false, true), + Arguments.of(Errors.UNKNOWN_TOPIC_ID, -1L, false, true), + Arguments.of(Errors.FETCH_SESSION_TOPIC_ID_ERROR, -1L, true, true), + Arguments.of(Errors.INCONSISTENT_TOPIC_ID, -1L, false, true), + Arguments.of(Errors.FENCED_LEADER_EPOCH, 100L, false, true), + Arguments.of(Errors.UNKNOWN_LEADER_EPOCH, 100L, false, false) + ); + } + + @Test + public void testEpochSetInFetchRequest() { + buildFetcher(); + subscriptions.assignFromUser(singleton(tp0)); + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWithIds("dummy", 1, + Collections.emptyMap(), Collections.singletonMap(topicName, 4), tp -> 99, topicIds); + client.updateMetadata(metadataResponse); + + subscriptions.seek(tp0, 10); + assertEquals(1, sendFetches()); + + // Check for epoch in outgoing request + MockClient.RequestMatcher matcher = body -> { + if (body instanceof FetchRequest) { + FetchRequest fetchRequest = (FetchRequest) body; + fetchRequest.fetchData(topicNames).values().forEach(partitionData -> { + assertTrue(partitionData.currentLeaderEpoch.isPresent(), "Expected Fetcher to set leader epoch in request"); + assertEquals(99, partitionData.currentLeaderEpoch.get().longValue(), "Expected leader epoch to match epoch from metadata update"); + }); + return true; + } else { + fail("Should have seen FetchRequest"); + return false; + } + }; + client.prepareResponse(matcher, fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.pollNoWakeup(); + assertEquals(0, networkClientDelegate.pendingRequestCount()); + } + + @Test + public void testFetchOffsetOutOfRange() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertEmptyFetch("Should not return records or advance position on fetch error"); + assertTrue(subscriptions.isOffsetResetNeeded(tp0)); + assertNull(subscriptions.validPosition(tp0)); + assertNull(subscriptions.position(tp0)); + } + + @Test + public void testStaleOutOfRangeError() { + // verify that an out of range error which arrives after a seek + // does not cause us to reset our position or throw an exception + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + subscriptions.seek(tp0, 1); + networkClientDelegate.poll(time.timer(0)); + assertEmptyFetch("Should not return records or advance position on fetch error"); + assertFalse(subscriptions.isOffsetResetNeeded(tp0)); + assertEquals(1, subscriptions.position(tp0).offset); + } + + @Test + public void testFetchedRecordsAfterSeek() { + buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), 2, IsolationLevel.READ_UNCOMMITTED); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertTrue(sendFetches() > 0); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + // The partition is not marked as needing its offset reset because that error handling logic is + // performed during the fetch collection. When we call seek() before we collect the fetch, the + // partition's position is updated (to offset 2) which is different from the offset from which + // we fetched the data (from offset 0). + assertFalse(subscriptions.isOffsetResetNeeded(tp0)); + subscriptions.seek(tp0, 2); + assertEmptyFetch("Should not return records or advance position after seeking to end of topic partition"); + } + + @Test + public void testFetchOffsetOutOfRangeException() { + buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), 2, IsolationLevel.READ_UNCOMMITTED); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + sendFetches(); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp0)); + for (int i = 0; i < 2; i++) { + OffsetOutOfRangeException e = assertThrows(OffsetOutOfRangeException.class, () -> + collectFetch()); + assertEquals(singleton(tp0), e.offsetOutOfRangePartitions().keySet()); + assertEquals(0L, e.offsetOutOfRangePartitions().get(tp0).longValue()); + } + } + + @Test + public void testFetchPositionAfterException() { + // verify the advancement in the next fetch offset equals to the number of fetched records when + // some fetched partitions cause Exception. This ensures that consumer won't lose record upon exception + buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); + assignFromUser(mkSet(tp0, tp1)); + subscriptions.seek(tp0, 1); + subscriptions.seek(tp1, 1); + + assertEquals(1, sendFetches()); + + Map partitions = new LinkedHashMap<>(); + partitions.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setHighWatermark(100) + .setRecords(records)); + partitions.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()) + .setHighWatermark(100)); + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); + networkClientDelegate.poll(time.timer(0)); + + List> allFetchedRecords = new ArrayList<>(); + fetchRecordsInto(allFetchedRecords); + + assertEquals(1, subscriptions.position(tp0).offset); + assertEquals(4, subscriptions.position(tp1).offset); + assertEquals(3, allFetchedRecords.size()); + + OffsetOutOfRangeException e = assertThrows(OffsetOutOfRangeException.class, () -> + fetchRecordsInto(allFetchedRecords)); + + assertEquals(singleton(tp0), e.offsetOutOfRangePartitions().keySet()); + assertEquals(1L, e.offsetOutOfRangePartitions().get(tp0).longValue()); + + assertEquals(1, subscriptions.position(tp0).offset); + assertEquals(4, subscriptions.position(tp1).offset); + assertEquals(3, allFetchedRecords.size()); + } + + private void fetchRecordsInto(List> allFetchedRecords) { + Map>> fetchedRecords = fetchRecords(); + fetchedRecords.values().forEach(allFetchedRecords::addAll); + } + + @Test + public void testCompletedFetchRemoval() { + // Ensure the removal of completed fetches that cause an Exception if and only if they contain empty records. + buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); + assignFromUser(mkSet(tp0, tp1, tp2, tp3)); + + subscriptions.seek(tp0, 1); + subscriptions.seek(tp1, 1); + subscriptions.seek(tp2, 1); + subscriptions.seek(tp3, 1); + + assertEquals(1, sendFetches()); + + Map partitions = new LinkedHashMap<>(); + partitions.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setHighWatermark(100) + .setRecords(records)); + partitions.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()) + .setHighWatermark(100)); + partitions.put(tidp2, new FetchResponseData.PartitionData() + .setPartitionIndex(tp2.partition()) + .setHighWatermark(100) + .setLastStableOffset(4) + .setLogStartOffset(0) + .setRecords(nextRecords)); + partitions.put(tidp3, new FetchResponseData.PartitionData() + .setPartitionIndex(tp3.partition()) + .setHighWatermark(100) + .setLastStableOffset(4) + .setLogStartOffset(0) + .setRecords(partialRecords)); + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); + networkClientDelegate.poll(time.timer(0)); + + List> fetchedRecords = new ArrayList<>(); + fetchRecordsInto(fetchedRecords); + + assertEquals(fetchedRecords.size(), subscriptions.position(tp1).offset - 1); + assertEquals(4, subscriptions.position(tp1).offset); + assertEquals(3, fetchedRecords.size()); + + List oorExceptions = new ArrayList<>(); + try { + fetchRecordsInto(fetchedRecords); + } catch (OffsetOutOfRangeException oor) { + oorExceptions.add(oor); + } + + // Should have received one OffsetOutOfRangeException for partition tp1 + assertEquals(1, oorExceptions.size()); + OffsetOutOfRangeException oor = oorExceptions.get(0); + assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0)); + assertEquals(oor.offsetOutOfRangePartitions().size(), 1); + + fetchRecordsInto(fetchedRecords); + + // Should not have received an Exception for tp2. + assertEquals(6, subscriptions.position(tp2).offset); + assertEquals(5, fetchedRecords.size()); + + int numExceptionsExpected = 3; + List kafkaExceptions = new ArrayList<>(); + for (int i = 1; i <= numExceptionsExpected; i++) { + try { + fetchRecordsInto(fetchedRecords); + } catch (KafkaException e) { + kafkaExceptions.add(e); + } + } + // Should have received as much as numExceptionsExpected Kafka exceptions for tp3. + assertEquals(numExceptionsExpected, kafkaExceptions.size()); + } + + @Test + public void testSeekBeforeException() { + buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), 2, IsolationLevel.READ_UNCOMMITTED); + + assignFromUser(mkSet(tp0)); + subscriptions.seek(tp0, 1); + assertEquals(1, sendFetches()); + Map partitions = new HashMap<>(); + partitions.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setHighWatermark(100) + .setRecords(records)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + + assertEquals(2, fetchRecords().get(tp0).size()); + + subscriptions.assignFromUser(mkSet(tp0, tp1)); + subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); + + assertEquals(1, sendFetches()); + partitions = new HashMap<>(); + partitions.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()) + .setHighWatermark(100)); + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); + networkClientDelegate.poll(time.timer(0)); + assertEquals(1, fetchRecords().get(tp0).size()); + + subscriptions.seek(tp1, 10); + // Should not throw OffsetOutOfRangeException after the seek + assertEmptyFetch("Should not return records or advance position after seeking to end of topic partitions"); + } + + @Test + public void testFetchDisconnected() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); + networkClientDelegate.poll(time.timer(0)); + assertEmptyFetch("Should not return records or advance position on disconnect"); + + // disconnects should have no effect on subscription state + assertFalse(subscriptions.isOffsetResetNeeded(tp0)); + assertTrue(subscriptions.isFetchable(tp0)); + assertEquals(0, subscriptions.position(tp0).offset); + } + + /* + * Send multiple requests. Verify that the client side quota metrics have the right values + */ + @Test + public void testQuotaMetrics() { + buildFetcher(); + + MockSelector selector = new MockSelector(time); + Cluster cluster = TestUtils.singletonCluster("test", 1); + Node node = cluster.nodes().get(0); + NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, + 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, + time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext()); + + ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( + 400, ApiMessageType.ListenerType.ZK_BROKER); + ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(apiVersionsResponse, ApiKeys.API_VERSIONS.latestVersion(), 0); + + selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); + while (!client.ready(node, time.milliseconds())) { + client.poll(1, time.milliseconds()); + // If a throttled response is received, advance the time to ensure progress. + time.sleep(client.throttleDelayMs(node, time.milliseconds())); + } + selector.clear(); + + for (int i = 1; i <= 3; i++) { + int throttleTimeMs = 100 * i; + FetchRequest.Builder builder = FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion(), 100, 100, new LinkedHashMap<>()); + builder.rackId(""); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); + client.send(request, time.milliseconds()); + client.poll(1, time.milliseconds()); + FetchResponse response = fullFetchResponse(tidp0, nextRecords, Errors.NONE, i, throttleTimeMs); + buffer = RequestTestUtils.serializeResponseWithHeader(response, ApiKeys.FETCH.latestVersion(), request.correlationId()); + selector.completeReceive(new NetworkReceive(node.idString(), buffer)); + client.poll(1, time.milliseconds()); + // If a throttled response is received, advance the time to ensure progress. + time.sleep(client.throttleDelayMs(node, time.milliseconds())); + selector.clear(); + } + Map allMetrics = metrics.metrics(); + KafkaMetric avgMetric = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchThrottleTimeAvg)); + KafkaMetric maxMetric = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchThrottleTimeMax)); + // Throttle times are ApiVersions=400, Fetch=(100, 200, 300) + assertEquals(250, (Double) avgMetric.metricValue(), EPSILON); + assertEquals(400, (Double) maxMetric.metricValue(), EPSILON); + client.close(); + } + + /* + * Send multiple requests. Verify that the client side quota metrics have the right values + */ + @Test + public void testFetcherMetrics() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + MetricName maxLagMetric = metrics.metricInstance(metricsRegistry.recordsLagMax); + Map tags = new HashMap<>(); + tags.put("topic", tp0.topic()); + tags.put("partition", String.valueOf(tp0.partition())); + MetricName partitionLagMetric = metrics.metricName("records-lag", metricGroup, tags); + + Map allMetrics = metrics.metrics(); + KafkaMetric recordsFetchLagMax = allMetrics.get(maxLagMetric); + + // recordsFetchLagMax should be initialized to NaN + assertEquals(Double.NaN, (Double) recordsFetchLagMax.metricValue(), EPSILON); + + // recordsFetchLagMax should be hw - fetchOffset after receiving an empty FetchResponse + fetchRecords(tidp0, MemoryRecords.EMPTY, Errors.NONE, 100L, 0); + assertEquals(100, (Double) recordsFetchLagMax.metricValue(), EPSILON); + + KafkaMetric partitionLag = allMetrics.get(partitionLagMetric); + assertEquals(100, (Double) partitionLag.metricValue(), EPSILON); + + // recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + fetchRecords(tidp0, builder.build(), Errors.NONE, 200L, 0); + assertEquals(197, (Double) recordsFetchLagMax.metricValue(), EPSILON); + assertEquals(197, (Double) partitionLag.metricValue(), EPSILON); + + // verify de-registration of partition lag + subscriptions.unsubscribe(); + sendFetches(); + assertFalse(allMetrics.containsKey(partitionLagMetric)); + } + + @Test + public void testFetcherLeadMetric() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + MetricName minLeadMetric = metrics.metricInstance(metricsRegistry.recordsLeadMin); + Map tags = new HashMap<>(2); + tags.put("topic", tp0.topic()); + tags.put("partition", String.valueOf(tp0.partition())); + MetricName partitionLeadMetric = metrics.metricName("records-lead", metricGroup, "", tags); + + Map allMetrics = metrics.metrics(); + KafkaMetric recordsFetchLeadMin = allMetrics.get(minLeadMetric); + + // recordsFetchLeadMin should be initialized to NaN + assertEquals(Double.NaN, (Double) recordsFetchLeadMin.metricValue(), EPSILON); + + // recordsFetchLeadMin should be position - logStartOffset after receiving an empty FetchResponse + fetchRecords(tidp0, MemoryRecords.EMPTY, Errors.NONE, 100L, -1L, 0L, 0); + assertEquals(0L, (Double) recordsFetchLeadMin.metricValue(), EPSILON); + + KafkaMetric partitionLead = allMetrics.get(partitionLeadMetric); + assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON); + + // recordsFetchLeadMin should be position - logStartOffset after receiving a non-empty FetchResponse + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) { + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + } + fetchRecords(tidp0, builder.build(), Errors.NONE, 200L, -1L, 0L, 0); + assertEquals(0L, (Double) recordsFetchLeadMin.metricValue(), EPSILON); + assertEquals(3L, (Double) partitionLead.metricValue(), EPSILON); + + // verify de-registration of partition lag + subscriptions.unsubscribe(); + sendFetches(); + assertFalse(allMetrics.containsKey(partitionLeadMetric)); + } + + @Test + public void testReadCommittedLagMetric() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + MetricName maxLagMetric = metrics.metricInstance(metricsRegistry.recordsLagMax); + + Map tags = new HashMap<>(); + tags.put("topic", tp0.topic()); + tags.put("partition", String.valueOf(tp0.partition())); + MetricName partitionLagMetric = metrics.metricName("records-lag", metricGroup, tags); + + Map allMetrics = metrics.metrics(); + KafkaMetric recordsFetchLagMax = allMetrics.get(maxLagMetric); + + // recordsFetchLagMax should be initialized to NaN + assertEquals(Double.NaN, (Double) recordsFetchLagMax.metricValue(), EPSILON); + + // recordsFetchLagMax should be lso - fetchOffset after receiving an empty FetchResponse + fetchRecords(tidp0, MemoryRecords.EMPTY, Errors.NONE, 100L, 50L, 0); + assertEquals(50, (Double) recordsFetchLagMax.metricValue(), EPSILON); + + KafkaMetric partitionLag = allMetrics.get(partitionLagMetric); + assertEquals(50, (Double) partitionLag.metricValue(), EPSILON); + + // recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + fetchRecords(tidp0, builder.build(), Errors.NONE, 200L, 150L, 0); + assertEquals(147, (Double) recordsFetchLagMax.metricValue(), EPSILON); + assertEquals(147, (Double) partitionLag.metricValue(), EPSILON); + + // verify de-registration of partition lag + subscriptions.unsubscribe(); + sendFetches(); + assertFalse(allMetrics.containsKey(partitionLagMetric)); + } + + @Test + public void testFetchResponseMetrics() { + buildFetcher(); + + String topic1 = "foo"; + String topic2 = "bar"; + TopicPartition tp1 = new TopicPartition(topic1, 0); + TopicPartition tp2 = new TopicPartition(topic2, 0); + + subscriptions.assignFromUser(mkSet(tp1, tp2)); + + Map partitionCounts = new HashMap<>(); + partitionCounts.put(topic1, 1); + partitionCounts.put(topic2, 1); + topicIds.put(topic1, Uuid.randomUuid()); + topicIds.put(topic2, Uuid.randomUuid()); + TopicIdPartition tidp1 = new TopicIdPartition(topicIds.get(topic1), tp1); + TopicIdPartition tidp2 = new TopicIdPartition(topicIds.get(topic2), tp2); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, partitionCounts, tp -> validLeaderEpoch, topicIds)); + + int expectedBytes = 0; + LinkedHashMap fetchPartitionData = new LinkedHashMap<>(); + + for (TopicIdPartition tp : mkSet(tidp1, tidp2)) { + subscriptions.seek(tp.topicPartition(), 0); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + MemoryRecords records = builder.build(); + for (Record record : records.records()) + expectedBytes += record.sizeInBytes(); + + fetchPartitionData.put(tp, new FetchResponseData.PartitionData() + .setPartitionIndex(tp.topicPartition().partition()) + .setHighWatermark(15) + .setLogStartOffset(0) + .setRecords(records)); + } + + assertEquals(1, sendFetches()); + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, fetchPartitionData)); + networkClientDelegate.poll(time.timer(0)); + + Map>> fetchedRecords = fetchRecords(); + assertEquals(3, fetchedRecords.get(tp1).size()); + assertEquals(3, fetchedRecords.get(tp2).size()); + + Map allMetrics = metrics.metrics(); + KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); + KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); + assertEquals(expectedBytes, (Double) fetchSizeAverage.metricValue(), EPSILON); + assertEquals(6, (Double) recordsCountAverage.metricValue(), EPSILON); + } + + @Test + public void testFetchResponseMetricsWithSkippedOffset() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 1); + + Map allMetrics = metrics.metrics(); + KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); + KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + MemoryRecords records = builder.build(); + + int expectedBytes = 0; + for (Record record : records.records()) { + if (record.offset() >= 1) + expectedBytes += record.sizeInBytes(); + } + + fetchRecords(tidp0, records, Errors.NONE, 100L, 0); + assertEquals(expectedBytes, (Double) fetchSizeAverage.metricValue(), EPSILON); + assertEquals(2, (Double) recordsCountAverage.metricValue(), EPSILON); + } + + @Test + public void testFetchResponseMetricsWithOnePartitionError() { + buildFetcher(); + assignFromUser(mkSet(tp0, tp1)); + subscriptions.seek(tp0, 0); + subscriptions.seek(tp1, 0); + + Map allMetrics = metrics.metrics(); + KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); + KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + MemoryRecords records = builder.build(); + + Map partitions = new HashMap<>(); + partitions.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setHighWatermark(100) + .setLogStartOffset(0) + .setRecords(records)); + partitions.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()) + .setHighWatermark(100) + .setLogStartOffset(0)); + + assertEquals(1, sendFetches()); + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); + networkClientDelegate.poll(time.timer(0)); + collectFetch(); + + int expectedBytes = 0; + for (Record record : records.records()) + expectedBytes += record.sizeInBytes(); + + assertEquals(expectedBytes, (Double) fetchSizeAverage.metricValue(), EPSILON); + assertEquals(3, (Double) recordsCountAverage.metricValue(), EPSILON); + } + + @Test + public void testFetchResponseMetricsWithOnePartitionAtTheWrongOffset() { + buildFetcher(); + + assignFromUser(mkSet(tp0, tp1)); + subscriptions.seek(tp0, 0); + subscriptions.seek(tp1, 0); + + Map allMetrics = metrics.metrics(); + KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); + KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); + + // send the fetch and then seek to a new offset + assertEquals(1, sendFetches()); + subscriptions.seek(tp1, 5); + + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, + TimestampType.CREATE_TIME, 0L); + for (int v = 0; v < 3; v++) + builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); + MemoryRecords records = builder.build(); + + Map partitions = new HashMap<>(); + partitions.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setHighWatermark(100) + .setLogStartOffset(0) + .setRecords(records)); + partitions.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setHighWatermark(100) + .setLogStartOffset(0) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("val".getBytes())))); + + client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); + networkClientDelegate.poll(time.timer(0)); + collectFetch(); + + // we should have ignored the record at the wrong offset + int expectedBytes = 0; + for (Record record : records.records()) + expectedBytes += record.sizeInBytes(); + + assertEquals(expectedBytes, (Double) fetchSizeAverage.metricValue(), EPSILON); + assertEquals(3, (Double) recordsCountAverage.metricValue(), EPSILON); + } + + @Test + public void testFetcherMetricsTemplates() { + Map clientTags = Collections.singletonMap("client-id", "clientA"); + buildFetcher(new MetricConfig().tags(clientTags), OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); + + // Fetch from topic to generate topic metrics + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + + // Verify that all metrics except metrics-count have registered templates + Set allMetrics = new HashSet<>(); + for (MetricName n : metrics.metrics().keySet()) { + String name = n.name().replaceAll(tp0.toString(), "{topic}-{partition}"); + if (!n.group().equals("kafka-metrics-count")) + allMetrics.add(new MetricNameTemplate(name, n.group(), "", n.tags().keySet())); + } + TestUtils.checkEquals(allMetrics, new HashSet<>(metricsRegistry.getAllTemplates()), "metrics", "templates"); + } + + private Map>> fetchRecords( + TopicIdPartition tp, MemoryRecords records, Errors error, long hw, int throttleTime) { + return fetchRecords(tp, records, error, hw, FetchResponse.INVALID_LAST_STABLE_OFFSET, throttleTime); + } + + private Map>> fetchRecords( + TopicIdPartition tp, MemoryRecords records, Errors error, long hw, long lastStableOffset, int throttleTime) { + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tp, records, error, hw, lastStableOffset, throttleTime)); + networkClientDelegate.poll(time.timer(0)); + return fetchRecords(); + } + + private Map>> fetchRecords( + TopicIdPartition tp, MemoryRecords records, Errors error, long hw, long lastStableOffset, long logStartOffset, int throttleTime) { + assertEquals(1, sendFetches()); + client.prepareResponse(fetchResponse(tp, records, error, hw, lastStableOffset, logStartOffset, throttleTime)); + networkClientDelegate.poll(time.timer(0)); + return fetchRecords(); + } + + @Test + public void testSkippingAbortedTransactions() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 0; + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); + + abortTransaction(buffer, 1L, currentOffset); + + buffer.flip(); + + List abortedTransactions = Collections.singletonList( + new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0)); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Fetch fetch = collectFetch(); + assertEquals(emptyMap(), fetch.records()); + assertTrue(fetch.positionAdvanced()); + } + + @Test + public void testReturnCommittedTransactions() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 0; + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); + + commitTransaction(buffer, 1L, currentOffset); + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + client.prepareResponse(body -> { + FetchRequest request = (FetchRequest) body; + assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel()); + return true; + }, fullFetchResponseWithAbortedTransactions(records, Collections.emptyList(), Errors.NONE, 100L, 100L, 0)); + + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + assertEquals(fetchedRecords.get(tp0).size(), 2); + } + + @Test + public void testReadCommittedWithCommittedAndAbortedTransactions() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + + List abortedTransactions = new ArrayList<>(); + + long pid1 = 1L; + long pid2 = 2L; + + // Appends for producer 1 (eventually committed) + appendTransactionalRecords(buffer, pid1, 0L, + new SimpleRecord("commit1-1".getBytes(), "value".getBytes()), + new SimpleRecord("commit1-2".getBytes(), "value".getBytes())); + + // Appends for producer 2 (eventually aborted) + appendTransactionalRecords(buffer, pid2, 2L, + new SimpleRecord("abort2-1".getBytes(), "value".getBytes())); + + // commit producer 1 + commitTransaction(buffer, pid1, 3L); + + // append more for producer 2 (eventually aborted) + appendTransactionalRecords(buffer, pid2, 4L, + new SimpleRecord("abort2-2".getBytes(), "value".getBytes())); + + // abort producer 2 + abortTransaction(buffer, pid2, 5L); + abortedTransactions.add(new FetchResponseData.AbortedTransaction().setProducerId(pid2).setFirstOffset(2L)); + + // New transaction for producer 1 (eventually aborted) + appendTransactionalRecords(buffer, pid1, 6L, + new SimpleRecord("abort1-1".getBytes(), "value".getBytes())); + + // New transaction for producer 2 (eventually committed) + appendTransactionalRecords(buffer, pid2, 7L, + new SimpleRecord("commit2-1".getBytes(), "value".getBytes())); + + // Add messages for producer 1 (eventually aborted) + appendTransactionalRecords(buffer, pid1, 8L, + new SimpleRecord("abort1-2".getBytes(), "value".getBytes())); + + // abort producer 1 + abortTransaction(buffer, pid1, 9L); + abortedTransactions.add(new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(6)); + + // commit producer 2 + commitTransaction(buffer, pid2, 10L); + + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + // There are only 3 committed records + List> fetchedConsumerRecords = fetchedRecords.get(tp0); + Set fetchedKeys = new HashSet<>(); + for (ConsumerRecord consumerRecord : fetchedConsumerRecords) { + fetchedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8)); + } + assertEquals(mkSet("commit1-1", "commit1-2", "commit2-1"), fetchedKeys); + } + + @Test + public void testMultipleAbortMarkers() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 0; + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "abort1-1".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "abort1-2".getBytes(), "value".getBytes())); + + currentOffset += abortTransaction(buffer, 1L, currentOffset); + // Duplicate abort -- should be ignored. + currentOffset += abortTransaction(buffer, 1L, currentOffset); + // Now commit a transaction. + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "commit1-1".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "commit1-2".getBytes(), "value".getBytes())); + commitTransaction(buffer, 1L, currentOffset); + buffer.flip(); + + List abortedTransactions = Collections.singletonList( + new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0) + ); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + assertEquals(fetchedRecords.get(tp0).size(), 2); + List> fetchedConsumerRecords = fetchedRecords.get(tp0); + Set expectedCommittedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2")); + Set actuallyCommittedKeys = new HashSet<>(); + for (ConsumerRecord consumerRecord : fetchedConsumerRecords) { + actuallyCommittedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8)); + } + assertEquals(expectedCommittedKeys, actuallyCommittedKeys); + } + + @Test + public void testReadCommittedAbortMarkerWithNoData() { + buildFetcher(OffsetResetStrategy.EARLIEST, new StringDeserializer(), + new StringDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + + long producerId = 1L; + + abortTransaction(buffer, producerId, 5L); + + appendTransactionalRecords(buffer, producerId, 6L, + new SimpleRecord("6".getBytes(), null), + new SimpleRecord("7".getBytes(), null), + new SimpleRecord("8".getBytes(), null)); + + commitTransaction(buffer, producerId, 9L); + + buffer.flip(); + + // send the fetch + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + + // prepare the response. the aborted transactions begin at offsets which are no longer in the log + List abortedTransactions = Collections.singletonList( + new FetchResponseData.AbortedTransaction().setProducerId(producerId).setFirstOffset(0L)); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer), + abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> allFetchedRecords = fetchRecords(); + assertTrue(allFetchedRecords.containsKey(tp0)); + List> fetchedRecords = allFetchedRecords.get(tp0); + assertEquals(3, fetchedRecords.size()); + assertEquals(Arrays.asList(6L, 7L, 8L), collectRecordOffsets(fetchedRecords)); + } + + @Test + public void testUpdatePositionWithLastRecordMissingFromBatch() { + buildFetcher(); + + MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord("0".getBytes(), "v".getBytes()), + new SimpleRecord("1".getBytes(), "v".getBytes()), + new SimpleRecord("2".getBytes(), "v".getBytes()), + new SimpleRecord(null, "value".getBytes())); + + // Remove the last record to simulate compaction + MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter(0, 0) { + @Override + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); + } + + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return record.key() != null; + } + }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + result.outputBuffer().flip(); + MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, compactedRecords, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> allFetchedRecords = fetchRecords(); + assertTrue(allFetchedRecords.containsKey(tp0)); + List> fetchedRecords = allFetchedRecords.get(tp0); + assertEquals(3, fetchedRecords.size()); + + for (int i = 0; i < 3; i++) { + assertEquals(Integer.toString(i), new String(fetchedRecords.get(i).key())); + } + + // The next offset should point to the next batch + assertEquals(4L, subscriptions.position(tp0).offset); + } + + @Test + public void testUpdatePositionOnEmptyBatch() { + buildFetcher(); + + long producerId = 1; + short producerEpoch = 0; + int sequence = 1; + long baseOffset = 37; + long lastOffset = 54; + int partitionLeaderEpoch = 7; + ByteBuffer buffer = ByteBuffer.allocate(DefaultRecordBatch.RECORD_BATCH_OVERHEAD); + DefaultRecordBatch.writeEmptyHeader(buffer, RecordBatch.CURRENT_MAGIC_VALUE, producerId, producerEpoch, + sequence, baseOffset, lastOffset, partitionLeaderEpoch, TimestampType.CREATE_TIME, + System.currentTimeMillis(), false, false); + buffer.flip(); + MemoryRecords recordsWithEmptyBatch = MemoryRecords.readableRecords(buffer); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, recordsWithEmptyBatch, Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Fetch fetch = collectFetch(); + assertEquals(emptyMap(), fetch.records()); + assertTrue(fetch.positionAdvanced()); + + // The next offset should point to the next batch + assertEquals(lastOffset + 1, subscriptions.position(tp0).offset); + } + + @Test + public void testReadCommittedWithCompactedTopic() { + buildFetcher(OffsetResetStrategy.EARLIEST, new StringDeserializer(), + new StringDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + + long pid1 = 1L; + long pid2 = 2L; + long pid3 = 3L; + + appendTransactionalRecords(buffer, pid3, 3L, + new SimpleRecord("3".getBytes(), "value".getBytes()), + new SimpleRecord("4".getBytes(), "value".getBytes())); + + appendTransactionalRecords(buffer, pid2, 15L, + new SimpleRecord("15".getBytes(), "value".getBytes()), + new SimpleRecord("16".getBytes(), "value".getBytes()), + new SimpleRecord("17".getBytes(), "value".getBytes())); + + appendTransactionalRecords(buffer, pid1, 22L, + new SimpleRecord("22".getBytes(), "value".getBytes()), + new SimpleRecord("23".getBytes(), "value".getBytes())); + + abortTransaction(buffer, pid2, 28L); + + appendTransactionalRecords(buffer, pid3, 30L, + new SimpleRecord("30".getBytes(), "value".getBytes()), + new SimpleRecord("31".getBytes(), "value".getBytes()), + new SimpleRecord("32".getBytes(), "value".getBytes())); + + commitTransaction(buffer, pid3, 35L); + + appendTransactionalRecords(buffer, pid1, 39L, + new SimpleRecord("39".getBytes(), "value".getBytes()), + new SimpleRecord("40".getBytes(), "value".getBytes())); + + // transaction from pid1 is aborted, but the marker is not included in the fetch + + buffer.flip(); + + // send the fetch + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + + // prepare the response. the aborted transactions begin at offsets which are no longer in the log + List abortedTransactions = Arrays.asList( + new FetchResponseData.AbortedTransaction().setProducerId(pid2).setFirstOffset(6), + new FetchResponseData.AbortedTransaction().setProducerId(pid1).setFirstOffset(0) + ); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer), + abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> allFetchedRecords = fetchRecords(); + assertTrue(allFetchedRecords.containsKey(tp0)); + List> fetchedRecords = allFetchedRecords.get(tp0); + assertEquals(5, fetchedRecords.size()); + assertEquals(Arrays.asList(3L, 4L, 30L, 31L, 32L), collectRecordOffsets(fetchedRecords)); + } + + @Test + public void testReturnAbortedTransactionsInUncommittedMode() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 0; + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); + + abortTransaction(buffer, 1L, currentOffset); + + buffer.flip(); + + List abortedTransactions = Collections.singletonList( + new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0)); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + } + + @Test + public void testConsumerPositionUpdatedWhenSkippingAbortedTransactions() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + long currentOffset = 0; + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "abort1-1".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "abort1-2".getBytes(), "value".getBytes())); + + currentOffset += abortTransaction(buffer, 1L, currentOffset); + buffer.flip(); + + List abortedTransactions = Collections.singletonList( + new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0)); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + + // Ensure that we don't return any of the aborted records, but yet advance the consumer position. + assertFalse(fetchedRecords.containsKey(tp0)); + assertEquals(currentOffset, subscriptions.position(tp0).offset); + } + + @Test + public void testConsumingViaIncrementalFetchRequests() { + buildFetcher(2); + + assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1))); + subscriptions.seekValidated(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), metadata.currentLeader(tp0))); + subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); + + // Fetch some records and establish an incremental fetch session. + LinkedHashMap partitions1 = new LinkedHashMap<>(); + partitions1.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setHighWatermark(2) + .setLastStableOffset(2) + .setLogStartOffset(0) + .setRecords(records)); + partitions1.put(tidp1, new FetchResponseData.PartitionData() + .setPartitionIndex(tp1.partition()) + .setHighWatermark(100) + .setLogStartOffset(0) + .setRecords(emptyRecords)); + FetchResponse resp1 = FetchResponse.of(Errors.NONE, 0, 123, partitions1); + client.prepareResponse(resp1); + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + Map>> fetchedRecords = fetchRecords(); + assertFalse(fetchedRecords.containsKey(tp1)); + List> recordsToTest = fetchedRecords.get(tp0); + assertEquals(2, recordsToTest.size()); + assertEquals(3L, subscriptions.position(tp0).offset); + assertEquals(1L, subscriptions.position(tp1).offset); + assertEquals(1, recordsToTest.get(0).offset()); + assertEquals(2, recordsToTest.get(1).offset()); + + // There is still a buffered record. + assertEquals(0, sendFetches()); + fetchedRecords = fetchRecords(); + assertFalse(fetchedRecords.containsKey(tp1)); + recordsToTest = fetchedRecords.get(tp0); + assertEquals(1, recordsToTest.size()); + assertEquals(3, recordsToTest.get(0).offset()); + assertEquals(4L, subscriptions.position(tp0).offset); + + // The second response contains no new records. + LinkedHashMap partitions2 = new LinkedHashMap<>(); + FetchResponse resp2 = FetchResponse.of(Errors.NONE, 0, 123, partitions2); + client.prepareResponse(resp2); + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.isEmpty()); + assertEquals(4L, subscriptions.position(tp0).offset); + assertEquals(1L, subscriptions.position(tp1).offset); + + // The third response contains some new records for tp0. + LinkedHashMap partitions3 = new LinkedHashMap<>(); + partitions3.put(tidp0, new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setHighWatermark(100) + .setLastStableOffset(4) + .setLogStartOffset(0) + .setRecords(nextRecords)); + FetchResponse resp3 = FetchResponse.of(Errors.NONE, 0, 123, partitions3); + client.prepareResponse(resp3); + assertEquals(1, sendFetches()); + networkClientDelegate.poll(time.timer(0)); + fetchedRecords = fetchRecords(); + assertFalse(fetchedRecords.containsKey(tp1)); + recordsToTest = fetchedRecords.get(tp0); + assertEquals(2, recordsToTest.size()); + assertEquals(6L, subscriptions.position(tp0).offset); + assertEquals(1L, subscriptions.position(tp1).offset); + assertEquals(4, recordsToTest.get(0).offset()); + assertEquals(5, recordsToTest.get(1).offset()); + } + + @Test + public void testEmptyControlBatch() { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 1; + + // Empty control batch should not cause an exception + DefaultRecordBatch.writeEmptyHeader(buffer, RecordBatch.MAGIC_VALUE_V2, 1L, + (short) 0, -1, 0, 0, + RecordBatch.NO_PARTITION_LEADER_EPOCH, TimestampType.CREATE_TIME, time.milliseconds(), + true, true); + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); + + commitTransaction(buffer, 1L, currentOffset); + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + client.prepareResponse(body -> { + FetchRequest request = (FetchRequest) body; + assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel()); + return true; + }, fullFetchResponseWithAbortedTransactions(records, Collections.emptyList(), Errors.NONE, 100L, 100L, 0)); + + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetchRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + assertEquals(fetchedRecords.get(tp0).size(), 2); + } + + private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset); + for (int i = 0; i < count; i++) + builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes()); + return builder.build(); + } + + private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) { + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true, + RecordBatch.NO_PARTITION_LEADER_EPOCH); + + for (SimpleRecord record : records) { + builder.append(record); + } + builder.build(); + return records.length; + } + + private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, SimpleRecord... records) { + return appendTransactionalRecords(buffer, pid, baseOffset, (int) baseOffset, records); + } + + private void commitTransaction(ByteBuffer buffer, long producerId, long baseOffset) { + short producerEpoch = 0; + int partitionLeaderEpoch = 0; + MemoryRecords.writeEndTransactionalMarker(buffer, baseOffset, time.milliseconds(), partitionLeaderEpoch, producerId, producerEpoch, + new EndTransactionMarker(ControlRecordType.COMMIT, 0)); + } + + private int abortTransaction(ByteBuffer buffer, long producerId, long baseOffset) { + short producerEpoch = 0; + int partitionLeaderEpoch = 0; + MemoryRecords.writeEndTransactionalMarker(buffer, baseOffset, time.milliseconds(), partitionLeaderEpoch, producerId, producerEpoch, + new EndTransactionMarker(ControlRecordType.ABORT, 0)); + return 1; + } + + @Test + public void testSubscriptionPositionUpdatedWithEpoch() { + // Create some records that include a leader epoch (1) + MemoryRecordsBuilder builder = MemoryRecords.builder( + ByteBuffer.allocate(1024), + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + 0L, + RecordBatch.NO_TIMESTAMP, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, + 1 + ); + builder.appendWithOffset(0L, 0L, "key".getBytes(), "value-1".getBytes()); + builder.appendWithOffset(1L, 0L, "key".getBytes(), "value-2".getBytes()); + builder.appendWithOffset(2L, 0L, "key".getBytes(), "value-3".getBytes()); + MemoryRecords records = builder.build(); + + buildFetcher(); + assignFromUser(singleton(tp0)); + + // Initialize the epoch=1 + Map partitionCounts = new HashMap<>(); + partitionCounts.put(tp0.topic(), 4); + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWithIds("dummy", 1, Collections.emptyMap(), partitionCounts, tp -> 1, topicIds); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L); + + // Seek + subscriptions.seek(tp0, 0); + + // Do a normal fetch + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + networkClientDelegate.pollNoWakeup(); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + + assertEquals(subscriptions.position(tp0).offset, 3L); + assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1)); + } + + @Test + public void testPreferredReadReplica() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + + // Take note of the preferred replica before the first fetch response + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> partitionRecords = fetchRecords(); + assertTrue(partitionRecords.containsKey(tp0)); + + // Verify + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Set preferred read replica to node=2, which isn't in our metadata, should revert to leader + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(2))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchDisconnectedShouldClearPreferredReadReplica() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Disconnect - preferred read replica should be cleared. + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); + + networkClientDelegate.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchDisconnectedShouldNotClearPreferredReadReplicaIfUnassigned() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Disconnect and remove tp0 from assignment + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); + subscriptions.assignFromUser(emptySet()); + + // Preferred read replica should not be cleared + networkClientDelegate.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchErrorShouldClearPreferredReadReplica() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Error - preferred read replica should be cleared. An actual error response will contain -1 as the + // preferred read replica. In the test we want to ensure that we are handling the error. + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.EMPTY, Errors.NOT_LEADER_OR_FOLLOWER, -1L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testPreferredReadReplicaOffsetError() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + fetchRecords(); + + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(selected.id(), 1); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Return an error, should unset the preferred read replica + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.empty())); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + fetchRecords(); + + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(selected.id(), -1); + } + + @Test + public void testFetchCompletedBeforeHandlerAdded() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + sendFetches(); + client.prepareResponse(fullFetchResponse(tidp0, buildRecords(1L, 1, 1), Errors.NONE, 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + fetchRecords(); + + Metadata.LeaderAndEpoch leaderAndEpoch = subscriptions.position(tp0).currentLeader; + assertTrue(leaderAndEpoch.leader.isPresent()); + Node readReplica = fetcher.selectReadReplica(tp0, leaderAndEpoch.leader.get(), time.milliseconds()); + + AtomicBoolean wokenUp = new AtomicBoolean(false); + client.setWakeupHook(() -> { + if (!wokenUp.getAndSet(true)) { + networkClientDelegate.disconnectAsync(readReplica); + networkClientDelegate.poll(time.timer(0)); + } + }); + + assertEquals(1, sendFetches()); + + networkClientDelegate.disconnectAsync(readReplica); + networkClientDelegate.poll(time.timer(0)); + + assertEquals(1, sendFetches()); + } + + @Test + public void testCorruptMessageError() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Prepare a response with the CORRUPT_MESSAGE error. + client.prepareResponse(fullFetchResponse( + tidp0, + buildRecords(1L, 1, 1), + Errors.CORRUPT_MESSAGE, + 100L, 0)); + networkClientDelegate.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + + // Trigger the exception. + assertThrows(KafkaException.class, this::fetchRecords); + } + + private OffsetsForLeaderEpochResponse prepareOffsetsForLeaderEpochResponse( + TopicPartition topicPartition, + Errors error, + int leaderEpoch, + long endOffset + ) { + OffsetForLeaderEpochResponseData data = new OffsetForLeaderEpochResponseData(); + data.topics().add(new OffsetForLeaderTopicResult() + .setTopic(topicPartition.topic()) + .setPartitions(Collections.singletonList(new EpochEndOffset() + .setPartition(topicPartition.partition()) + .setErrorCode(error.code()) + .setLeaderEpoch(leaderEpoch) + .setEndOffset(endOffset)))); + return new OffsetsForLeaderEpochResponse(data); + } + + private FetchResponse fetchResponseWithTopLevelError(TopicIdPartition tp, Errors error, int throttleTime) { + Map partitions = Collections.singletonMap(tp, + new FetchResponseData.PartitionData() + .setPartitionIndex(tp.topicPartition().partition()) + .setErrorCode(error.code()) + .setHighWatermark(FetchResponse.INVALID_HIGH_WATERMARK)); + return FetchResponse.of(error, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)); + } + + private FetchResponse fullFetchResponseWithAbortedTransactions(MemoryRecords records, + List abortedTransactions, + Errors error, + long lastStableOffset, + long hw, + int throttleTime) { + Map partitions = Collections.singletonMap(tidp0, + new FetchResponseData.PartitionData() + .setPartitionIndex(tp0.partition()) + .setErrorCode(error.code()) + .setHighWatermark(hw) + .setLastStableOffset(lastStableOffset) + .setLogStartOffset(0) + .setAbortedTransactions(abortedTransactions) + .setRecords(records)); + return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)); + } + + private FetchResponse fullFetchResponse(int sessionId, TopicIdPartition tp, MemoryRecords records, Errors error, long hw, int throttleTime) { + return fullFetchResponse(sessionId, tp, records, error, hw, FetchResponse.INVALID_LAST_STABLE_OFFSET, throttleTime); + } + + private FetchResponse fullFetchResponse(TopicIdPartition tp, MemoryRecords records, Errors error, long hw, int throttleTime) { + return fullFetchResponse(tp, records, error, hw, FetchResponse.INVALID_LAST_STABLE_OFFSET, throttleTime); + } + + private FetchResponse fullFetchResponse(TopicIdPartition tp, MemoryRecords records, Errors error, long hw, + long lastStableOffset, int throttleTime) { + return fullFetchResponse(INVALID_SESSION_ID, tp, records, error, hw, lastStableOffset, throttleTime); + } + + private FetchResponse fullFetchResponse(int sessionId, TopicIdPartition tp, MemoryRecords records, Errors error, long hw, + long lastStableOffset, int throttleTime) { + Map partitions = Collections.singletonMap(tp, + new FetchResponseData.PartitionData() + .setPartitionIndex(tp.topicPartition().partition()) + .setErrorCode(error.code()) + .setHighWatermark(hw) + .setLastStableOffset(lastStableOffset) + .setLogStartOffset(0) + .setRecords(records)); + return FetchResponse.of(Errors.NONE, throttleTime, sessionId, new LinkedHashMap<>(partitions)); + } + + private FetchResponse fullFetchResponse(TopicIdPartition tp, MemoryRecords records, Errors error, long hw, + long lastStableOffset, int throttleTime, Optional preferredReplicaId) { + Map partitions = Collections.singletonMap(tp, + new FetchResponseData.PartitionData() + .setPartitionIndex(tp.topicPartition().partition()) + .setErrorCode(error.code()) + .setHighWatermark(hw) + .setLastStableOffset(lastStableOffset) + .setLogStartOffset(0) + .setRecords(records) + .setPreferredReadReplica(preferredReplicaId.orElse(FetchResponse.INVALID_PREFERRED_REPLICA_ID))); + return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)); + } + + private FetchResponse fetchResponse(TopicIdPartition tp, MemoryRecords records, Errors error, long hw, + long lastStableOffset, long logStartOffset, int throttleTime) { + Map partitions = Collections.singletonMap(tp, + new FetchResponseData.PartitionData() + .setPartitionIndex(tp.topicPartition().partition()) + .setErrorCode(error.code()) + .setHighWatermark(hw) + .setLastStableOffset(lastStableOffset) + .setLogStartOffset(logStartOffset) + .setRecords(records)); + return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)); + } + + /** + * Assert that the {@link Fetcher#collectFetch() latest fetch} does not contain any + * {@link Fetch#records() user-visible records}, did not + * {@link Fetch#positionAdvanced() advance the consumer's position}, + * and is {@link Fetch#isEmpty() empty}. + * @param reason the reason to include for assertion methods such as {@link org.junit.jupiter.api.Assertions#assertTrue(boolean, String)} + */ + private void assertEmptyFetch(String reason) { + Fetch fetch = collectFetch(); + assertEquals(Collections.emptyMap(), fetch.records(), reason); + assertFalse(fetch.positionAdvanced(), reason); + assertTrue(fetch.isEmpty(), reason); + } + + private Map>> fetchRecords() { + Fetch fetch = collectFetch(); + return fetch.records(); + } + + @SuppressWarnings("unchecked") + private Fetch collectFetch() { + return (Fetch) fetcher.collectFetch(); + } + + private void buildFetcher(int maxPollRecords) { + buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), new ByteArrayDeserializer(), + maxPollRecords, IsolationLevel.READ_UNCOMMITTED); + } + + private void buildFetcher() { + buildFetcher(Integer.MAX_VALUE); + } + + private void buildFetcher(Deserializer keyDeserializer, + Deserializer valueDeserializer) { + buildFetcher(OffsetResetStrategy.EARLIEST, keyDeserializer, valueDeserializer, + Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); + } + + private void buildFetcher(OffsetResetStrategy offsetResetStrategy, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + int maxPollRecords, + IsolationLevel isolationLevel) { + buildFetcher(new MetricConfig(), offsetResetStrategy, keyDeserializer, valueDeserializer, + maxPollRecords, isolationLevel); + } + + private void buildFetcher(MetricConfig metricConfig, + OffsetResetStrategy offsetResetStrategy, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + int maxPollRecords, + IsolationLevel isolationLevel) { + buildFetcher(metricConfig, offsetResetStrategy, keyDeserializer, valueDeserializer, maxPollRecords, isolationLevel, Long.MAX_VALUE); + } + + private void buildFetcher(MetricConfig metricConfig, + OffsetResetStrategy offsetResetStrategy, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + int maxPollRecords, + IsolationLevel isolationLevel, + long metadataExpireMs) { + LogContext logContext = new LogContext(); + SubscriptionState subscriptionState = new SubscriptionState(logContext, offsetResetStrategy); + buildFetcher(metricConfig, keyDeserializer, valueDeserializer, maxPollRecords, isolationLevel, metadataExpireMs, + subscriptionState, logContext); + } + + private void buildFetcher(MetricConfig metricConfig, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + int maxPollRecords, + IsolationLevel isolationLevel, + long metadataExpireMs, + SubscriptionState subscriptionState, + LogContext logContext) { + buildDependencies(metricConfig, metadataExpireMs, subscriptionState, logContext); + Deserializers deserializers = new Deserializers<>(keyDeserializer, valueDeserializer); + FetchConfig fetchConfig = new FetchConfig( + minBytes, + maxBytes, + maxWaitMs, + fetchSize, + maxPollRecords, + true, // check crc + CommonClientConfigs.DEFAULT_CLIENT_RACK, + isolationLevel); + FetchCollector fetchCollector = new FetchCollector<>(logContext, + metadata, + subscriptions, + fetchConfig, + deserializers, + metricsManager, + time); + fetcher = spy(new TestableFetchRequestManager<>( + logContext, + time, + metadata, + subscriptionState, + fetchConfig, + new FetchBuffer(logContext), + metricsManager, + networkClientDelegate, + fetchCollector)); + ConsumerNetworkClient consumerNetworkClient = new ConsumerNetworkClient( + logContext, + client, + metadata, + time, + retryBackoffMs, + (int) requestTimeoutMs, + Integer.MAX_VALUE); + offsetFetcher = new OffsetFetcher(logContext, + consumerNetworkClient, + metadata, + subscriptions, + time, + retryBackoffMs, + requestTimeoutMs, + isolationLevel, + apiVersions); + } + + private void buildDependencies(MetricConfig metricConfig, + long metadataExpireMs, + SubscriptionState subscriptionState, + LogContext logContext) { + time = new MockTime(1, 0, 0); + subscriptions = subscriptionState; + metadata = new ConsumerMetadata(0, 0, metadataExpireMs, false, false, + subscriptions, logContext, new ClusterResourceListeners()); + client = new MockClient(time, metadata); + metrics = new Metrics(metricConfig, time); + metricsRegistry = new FetchMetricsRegistry(metricConfig.tags().keySet(), "consumer" + groupId); + metricsManager = new FetchMetricsManager(metrics, metricsRegistry); + + Properties properties = new Properties(); + properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(requestTimeoutMs)); + properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); + ConsumerConfig config = new ConsumerConfig(properties); + networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client)); + } + + private List collectRecordOffsets(List> records) { + return records.stream().map(ConsumerRecord::offset).collect(Collectors.toList()); + } + + private class TestableFetchRequestManager extends FetchRequestManager { + + private final FetchCollector fetchCollector; + + public TestableFetchRequestManager(LogContext logContext, + Time time, + ConsumerMetadata metadata, + SubscriptionState subscriptions, + FetchConfig fetchConfig, + FetchBuffer fetchBuffer, + FetchMetricsManager metricsManager, + NetworkClientDelegate networkClientDelegate, + FetchCollector fetchCollector) { + super(logContext, time, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, networkClientDelegate); + this.fetchCollector = fetchCollector; + } + + private Fetch collectFetch() { + return fetchCollector.collectFetch(fetchBuffer); + } + + private int sendFetches() { + NetworkClientDelegate.PollResult pollResult = poll(time.milliseconds()); + networkClientDelegate.addAll(pollResult.unsentRequests); + return pollResult.unsentRequests.size(); + } + + private void clearBufferedDataForUnassignedPartitions(Set partitions) { + fetchBuffer.retainAll(partitions); + } + } + + private class TestableNetworkClientDelegate extends NetworkClientDelegate { + + private final Logger log = LoggerFactory.getLogger(NetworkClientDelegate.class); + private final ConcurrentLinkedQueue pendingDisconnects = new ConcurrentLinkedQueue<>(); + + public TestableNetworkClientDelegate(Time time, + ConsumerConfig config, + LogContext logContext, + KafkaClient client) { + super(time, config, logContext, client); + } + + @Override + public void poll(final long timeoutMs, final long currentTimeMs) { + handlePendingDisconnects(); + super.poll(timeoutMs, currentTimeMs); + } + + public void pollNoWakeup() { + poll(time.timer(0)); + } + + public int pendingRequestCount() { + return unsentRequests().size() + client.inFlightRequestCount(); + } + + public void poll(final Timer timer) { + long pollTimeout = Math.min(timer.remainingMs(), requestTimeoutMs); + if (client.inFlightRequestCount() == 0) + pollTimeout = Math.min(pollTimeout, retryBackoffMs); + poll(pollTimeout, timer.currentTimeMs()); + } + + private Set unsentRequestNodes() { + Set set = new HashSet<>(); + + for (UnsentRequest u : unsentRequests()) + u.node().ifPresent(set::add); + + return set; + } + + private List removeUnsentRequestByNode(Node node) { + List list = new ArrayList<>(); + + Iterator iter = unsentRequests().iterator(); + + while (iter.hasNext()) { + UnsentRequest u = iter.next(); + + if (node.equals(u.node().orElse(null))) { + iter.remove(); + list.add(u); + } + } + + return list; + } + + @Override + protected void checkDisconnects(final long currentTimeMs) { + // any disconnects affecting requests that have already been transmitted will be handled + // by NetworkClient, so we just need to check whether connections for any of the unsent + // requests have been disconnected; if they have, then we complete the corresponding future + // and set the disconnect flag in the ClientResponse + for (Node node : unsentRequestNodes()) { + if (client.connectionFailed(node)) { + // Remove entry before invoking request callback to avoid callbacks handling + // coordinator failures traversing the unsent list again. + for (UnsentRequest unsentRequest : removeUnsentRequestByNode(node)) { + // TODO: this should likely emulate what's done in ConsumerNetworkClient + log.error("checkDisconnects - please update! unsentRequest: {}", unsentRequest); + } + } + } + } + + private void handlePendingDisconnects() { + while (true) { + Node node = pendingDisconnects.poll(); + if (node == null) + break; + + failUnsentRequests(node, DisconnectException.INSTANCE); + client.disconnect(node.idString()); + } + } + + public void disconnectAsync(Node node) { + pendingDisconnects.offer(node); + client.wakeup(); + } + + private void failUnsentRequests(Node node, RuntimeException e) { + // clear unsent requests to node and fail their corresponding futures + for (UnsentRequest unsentRequest : removeUnsentRequestByNode(node)) { + FutureCompletionHandler handler = unsentRequest.handler(); + handler.onFailure(time.milliseconds(), e); + } + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 6336c11219e..a66d153f098 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -90,6 +90,9 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; import java.io.DataOutputStream; @@ -116,6 +119,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -209,22 +213,22 @@ public class FetcherTest { tp -> validLeaderEpoch, topicIds), false, 0L); } - private void assignFromUserNoId(Set partitions) { - subscriptions.assignFromUser(partitions); - client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singletonMap("noId", 1), Collections.emptyMap())); + private void assignFromUser(TopicPartition partition) { + subscriptions.assignFromUser(singleton(partition)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singletonMap(partition.topic(), 1), Collections.emptyMap())); // A dummy metadata update to ensure valid leader epoch. metadata.update(9, RequestTestUtils.metadataUpdateWithIds("dummy", 1, - Collections.emptyMap(), singletonMap("noId", 1), + Collections.emptyMap(), singletonMap(partition.topic(), 1), tp -> validLeaderEpoch, topicIds), false, 0L); } @AfterEach public void teardown() throws Exception { if (metrics != null) - this.metrics.close(); + metrics.close(); if (fetcher != null) - this.fetcher.close(); + fetcher.close(); if (executorService != null) { executorService.shutdownNow(); assertTrue(executorService.awaitTermination(5, TimeUnit.SECONDS)); @@ -247,11 +251,11 @@ public class FetcherTest { assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); List> records = partitionRecords.get(tp0); @@ -274,9 +278,9 @@ public class FetcherTest { assertEquals(1, sendFetches()); subscriptions.markPendingRevocation(singleton(tp0)); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - assertNull(fetchedRecords().get(tp0)); + assertNull(fetchRecords().get(tp0)); } @Test @@ -301,7 +305,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - final FetchResponse fetchResponse = fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0); + final FetchResponse fetchResponse = fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0); client.prepareResponse(fetchResponse); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); @@ -310,7 +314,7 @@ public class FetcherTest { final ArgumentCaptor argument = ArgumentCaptor.forClass(FetchRequest.Builder.class); // send request to close the fetcher - this.fetcher.close(time.timer(Duration.ofSeconds(10))); + fetcher.close(time.timer(Duration.ofSeconds(10))); // validate that Fetcher.close() has sent a request with final epoch. 2 requests are sent, one for the normal // fetch earlier and another for the finish fetch here. @@ -332,10 +336,10 @@ public class FetcherTest { // normal fetch assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); assertEquals(4L, subscriptions.position(tp0).offset); // this is the next fetching position // mark partition unfetchable @@ -343,7 +347,7 @@ public class FetcherTest { assertEquals(0, sendFetches()); consumerClient.poll(time.timer(0)); assertFalse(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); assertEquals(4L, subscriptions.position(tp0).offset); } @@ -353,21 +357,21 @@ public class FetcherTest { buildFetcher(); TopicIdPartition noId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("noId", 0)); - assignFromUserNoId(singleton(noId.topicPartition())); + assignFromUser(noId.topicPartition()); subscriptions.seek(noId.topicPartition(), 0); - // Fetch should use request version 12 assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); + // Fetch should use request version 12 client.prepareResponse( fetchRequestMatcher((short) 12, noId, 0, Optional.of(validLeaderEpoch)), - fullFetchResponse(noId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(noId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(noId.topicPartition())); List> records = partitionRecords.get(noId.topicPartition()); @@ -388,18 +392,18 @@ public class FetcherTest { assignFromUser(singleton(tp.topicPartition())); subscriptions.seek(tp.topicPartition(), 0); - // Fetch should use latest version assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); + // Fetch should use latest version client.prepareResponse( fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), tp, 0, Optional.of(validLeaderEpoch)), - fullFetchResponse(tp, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(tp, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp.topicPartition())); List> records = partitionRecords.get(tp.topicPartition()); @@ -424,9 +428,9 @@ public class FetcherTest { client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(foo), tp -> validLeaderEpoch)); subscriptions.seek(foo.topicPartition(), 0); - // Fetch should use latest version. assertEquals(1, sendFetches()); + // Fetch should use latest version. client.prepareResponse( fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), singletonMap(foo, new PartitionData( @@ -438,13 +442,13 @@ public class FetcherTest { ), emptyList() ), - fullFetchResponse(1, foo, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, foo, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); - // Assign bar and unassign foo. + // Assign bar and un-assign foo. subscriptions.assignFromUser(singleton(bar.topicPartition())); client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(bar), tp -> validLeaderEpoch)); subscriptions.seek(bar.topicPartition(), 0); @@ -464,11 +468,11 @@ public class FetcherTest { ), singletonList(foo) ), - fullFetchResponse(1, bar, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, bar, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); } @Test @@ -497,11 +501,11 @@ public class FetcherTest { ), emptyList() ), - fullFetchResponse(1, fooWithOldTopicId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, fooWithOldTopicId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Replace foo with old topic id with foo with new topic id. subscriptions.assignFromUser(singleton(fooWithNewTopicId.topicPartition())); @@ -524,11 +528,11 @@ public class FetcherTest { ), singletonList(fooWithOldTopicId) ), - fullFetchResponse(1, fooWithNewTopicId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, fooWithNewTopicId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); } @Test @@ -536,7 +540,6 @@ public class FetcherTest { buildFetcher(); TopicIdPartition fooWithoutId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("foo", 0)); - TopicIdPartition fooWithId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); // Assign foo without a topic id. subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition())); @@ -557,13 +560,14 @@ public class FetcherTest { ), emptyList() ), - fullFetchResponse(1, fooWithoutId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, fooWithoutId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Upgrade. + TopicIdPartition fooWithId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); subscriptions.assignFromUser(singleton(fooWithId.topicPartition())); client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithId), tp -> validLeaderEpoch)); subscriptions.seek(fooWithId.topicPartition(), 0); @@ -584,11 +588,11 @@ public class FetcherTest { ), emptyList() ), - fullFetchResponse(1, fooWithId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, fooWithId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Downgrade. subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition())); @@ -611,11 +615,11 @@ public class FetcherTest { ), emptyList() ), - fullFetchResponse(1, fooWithoutId, this.records, Errors.NONE, 100L, 0) + fullFetchResponse(1, fooWithoutId, records, Errors.NONE, 100L, 0) ); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); } private MockClient.RequestMatcher fetchRequestMatcher( @@ -684,7 +688,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); assertEquals(2, partitionRecords.get(tp0).size()); @@ -735,7 +739,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); assertEquals(6, partitionRecords.get(tp0).size()); @@ -756,7 +760,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); Set newAssignedTopicPartitions = new HashSet<>(); @@ -812,7 +816,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); List> records = partitionRecords.get(tp0); @@ -833,11 +837,11 @@ public class FetcherTest { assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0)); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertFalse(partitionRecords.containsKey(tp0)); } @@ -872,19 +876,16 @@ public class FetcherTest { assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 1); - client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - // The fetcher should block on Deserialization error + for (int i = 0; i < 2; i++) { - try { - fetcher.collectFetch(); - fail("fetchedRecords should have raised"); - } catch (SerializationException e) { - // the position should not advance since no data has been returned - assertEquals(1, subscriptions.position(tp0).offset); - } + // The fetcher should throw a Deserialization error + assertThrows(SerializationException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(1, subscriptions.position(tp0).offset); } } @@ -939,8 +940,8 @@ public class FetcherTest { client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - // the first fetchedRecords() should return the first valid message - assertEquals(1, fetchedRecords().get(tp0).size()); + // the first fetchRecords() should return the first valid message + assertEquals(1, fetchRecords().get(tp0).size()); assertEquals(1, subscriptions.position(tp0).offset); ensureBlockOnRecord(1L); @@ -957,14 +958,10 @@ public class FetcherTest { } private void ensureBlockOnRecord(long blockedOffset) { - // the fetchedRecords() should always throw exception due to the invalid message at the starting offset. for (int i = 0; i < 2; i++) { - try { - fetcher.collectFetch(); - fail("fetchedRecords should have raised KafkaException"); - } catch (KafkaException e) { - assertEquals(blockedOffset, subscriptions.position(tp0).offset); - } + // the fetchRecords() should always throw exception due to the invalid message at the starting offset. + assertThrows(KafkaException.class, this::fetchRecords); + assertEquals(blockedOffset, subscriptions.position(tp0).offset); } } @@ -972,12 +969,12 @@ public class FetcherTest { // Seek to skip the bad record and fetch again. subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(toOffset, Optional.empty(), metadata.currentLeader(tp0))); // Should not throw exception after the seek. - fetcher.collectFetch(); + collectFetch(); assertEquals(1, sendFetches()); client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(responseBuffer), Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - Map>> recordsByPartition = fetchedRecords(); + Map>> recordsByPartition = fetchRecords(); List> records = recordsByPartition.get(tp0); assertEquals(1, records.size()); assertEquals(toOffset, records.get(0).offset()); @@ -1013,14 +1010,10 @@ public class FetcherTest { client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - // the fetchedRecords() should always throw exception due to the bad batch. for (int i = 0; i < 2; i++) { - try { - fetcher.collectFetch(); - fail("fetchedRecords should have raised KafkaException"); - } catch (KafkaException e) { - assertEquals(0, subscriptions.position(tp0).offset); - } + // collectFetch() should always throw exception due to the bad batch. + assertThrows(KafkaException.class, this::collectFetch); + assertEquals(0, subscriptions.position(tp0).offset); } } @@ -1044,13 +1037,10 @@ public class FetcherTest { assertEquals(1, sendFetches()); client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.readableRecords(buffer), Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - try { - fetcher.collectFetch(); - fail("fetchedRecords should have raised"); - } catch (KafkaException e) { - // the position should not advance since no data has been returned - assertEquals(0, subscriptions.position(tp0).offset); - } + + assertThrows(KafkaException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp0).offset); } @Test @@ -1079,7 +1069,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - Map>> recordsByPartition = fetchedRecords(); + Map>> recordsByPartition = fetchRecords(); records = recordsByPartition.get(tp0); assertEquals(3, records.size()); @@ -1102,38 +1092,37 @@ public class FetcherTest { public void testFetchMaxPollRecords() { buildFetcher(2); - List> records; assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 1); - client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); - client.prepareResponse(matchesOffset(tidp0, 4), fullFetchResponse(tidp0, this.nextRecords, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp0, 4), fullFetchResponse(tidp0, nextRecords, Errors.NONE, 100L, 0)); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - Map>> recordsByPartition = fetchedRecords(); - records = recordsByPartition.get(tp0); - assertEquals(2, records.size()); + Map>> recordsByPartition = fetchRecords(); + List> recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); assertEquals(3L, subscriptions.position(tp0).offset); - assertEquals(1, records.get(0).offset()); - assertEquals(2, records.get(1).offset()); + assertEquals(1, recordsToTest.get(0).offset()); + assertEquals(2, recordsToTest.get(1).offset()); assertEquals(0, sendFetches()); consumerClient.poll(time.timer(0)); - recordsByPartition = fetchedRecords(); - records = recordsByPartition.get(tp0); - assertEquals(1, records.size()); + recordsByPartition = fetchRecords(); + recordsToTest = recordsByPartition.get(tp0); + assertEquals(1, recordsToTest.size()); assertEquals(4L, subscriptions.position(tp0).offset); - assertEquals(3, records.get(0).offset()); + assertEquals(3, recordsToTest.get(0).offset()); assertTrue(sendFetches() > 0); consumerClient.poll(time.timer(0)); - recordsByPartition = fetchedRecords(); - records = recordsByPartition.get(tp0); - assertEquals(2, records.size()); + recordsByPartition = fetchRecords(); + recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); assertEquals(6L, subscriptions.position(tp0).offset); - assertEquals(4, records.get(0).offset()); - assertEquals(5, records.get(1).offset()); + assertEquals(4, recordsToTest.get(0).offset()); + assertEquals(5, recordsToTest.get(1).offset()); } /** @@ -1145,35 +1134,34 @@ public class FetcherTest { public void testFetchAfterPartitionWithFetchedRecordsIsUnassigned() { buildFetcher(2); - List> records; assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 1); // Returns 3 records while `max.poll.records` is configured to 2 - client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp0, 1), fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - Map>> recordsByPartition = fetchedRecords(); - records = recordsByPartition.get(tp0); - assertEquals(2, records.size()); + Map>> recordsByPartition = fetchRecords(); + List> recordsToTest = recordsByPartition.get(tp0); + assertEquals(2, recordsToTest.size()); assertEquals(3L, subscriptions.position(tp0).offset); - assertEquals(1, records.get(0).offset()); - assertEquals(2, records.get(1).offset()); + assertEquals(1, recordsToTest.get(0).offset()); + assertEquals(2, recordsToTest.get(1).offset()); assignFromUser(singleton(tp1)); - client.prepareResponse(matchesOffset(tidp1, 4), fullFetchResponse(tidp1, this.nextRecords, Errors.NONE, 100L, 0)); + client.prepareResponse(matchesOffset(tidp1, 4), fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); subscriptions.seek(tp1, 4); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertNull(fetchedRecords.get(tp0)); - records = fetchedRecords.get(tp1); - assertEquals(2, records.size()); + recordsToTest = fetchedRecords.get(tp1); + assertEquals(2, recordsToTest.size()); assertEquals(6L, subscriptions.position(tp1).offset); - assertEquals(4, records.get(0).offset()); - assertEquals(5, records.get(1).offset()); + assertEquals(4, recordsToTest.get(0).offset()); + assertEquals(5, recordsToTest.get(1).offset()); } @Test @@ -1197,7 +1185,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - Map>> recordsByPartition = fetchedRecords(); + Map>> recordsByPartition = fetchRecords(); consumerRecords = recordsByPartition.get(tp0); assertEquals(3, consumerRecords.size()); assertEquals(31L, subscriptions.position(tp0).offset); // this is the next fetching position @@ -1218,14 +1206,9 @@ public class FetcherTest { client.setNodeApiVersions(NodeApiVersions.create(ApiKeys.FETCH.id, (short) 2, (short) 2)); makeFetchRequestWithIncompleteRecord(); - try { - fetcher.collectFetch(); - fail("RecordTooLargeException should have been raised"); - } catch (RecordTooLargeException e) { - assertTrue(e.getMessage().startsWith("There are some messages at [Partition=Offset]: ")); - // the position should not advance since no data has been returned - assertEquals(0, subscriptions.position(tp0).offset); - } + assertThrows(RecordTooLargeException.class, this::collectFetch); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp0).offset); } finally { client.setNodeApiVersions(NodeApiVersions.create()); } @@ -1242,8 +1225,8 @@ public class FetcherTest { buildFetcher(); makeFetchRequestWithIncompleteRecord(); try { - fetcher.collectFetch(); - fail("RecordTooLargeException should have been raised"); + collectFetch(); + fail("collectFetch should have thrown a KafkaException"); } catch (KafkaException e) { assertTrue(e.getMessage().startsWith("Failed to make progress reading messages")); // the position should not advance since no data has been returned @@ -1270,13 +1253,12 @@ public class FetcherTest { assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 0); - // resize the limit of the buffer to pretend it is only fetch-size large assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.TOPIC_AUTHORIZATION_FAILED, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.TOPIC_AUTHORIZATION_FAILED, 100L, 0)); consumerClient.poll(time.timer(0)); try { - fetcher.collectFetch(); - fail("fetchedRecords should have thrown"); + collectFetch(); + fail("collectFetch should have thrown a TopicAuthorizationException"); } catch (TopicAuthorizationException e) { assertEquals(singleton(topicName), e.unauthorizedTopics()); } @@ -1299,11 +1281,11 @@ public class FetcherTest { subscriptions.assignFromSubscribed(Collections.emptyList()); subscriptions.assignFromSubscribed(singleton(tp0)); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); // The active fetch should be ignored since its position is no longer valid - assertTrue(fetchedRecords().isEmpty()); + assertTrue(fetchRecords().isEmpty()); } @Test @@ -1322,10 +1304,10 @@ public class FetcherTest { // Now the cooperative rebalance happens and fetch positions are NOT cleared for unrevoked partitions subscriptions.assignFromSubscribed(singleton(tp0)); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); // The active fetch should NOT be ignored since the position for tp0 is still valid assertEquals(1, fetchedRecords.size()); @@ -1342,9 +1324,9 @@ public class FetcherTest { assertEquals(1, sendFetches()); subscriptions.pause(tp0); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - assertNull(fetchedRecords().get(tp0)); + assertNull(fetchRecords().get(tp0)); } @Test @@ -1370,7 +1352,7 @@ public class FetcherTest { subscriptions.pause(tp0); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); assertEmptyFetch("Should not return any records or advance position when partition is paused"); @@ -1384,7 +1366,7 @@ public class FetcherTest { assertTrue(fetcher.hasAvailableFetches(), "Should have available (non-paused) completed fetches"); consumerClient.poll(time.timer(0)); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertEquals(1, fetchedRecords.size(), "Should return records when partition is resumed"); assertNotNull(fetchedRecords.get(tp0)); assertEquals(3, fetchedRecords.get(tp0).size()); @@ -1407,18 +1389,18 @@ public class FetcherTest { // #1 seek, request, poll, response subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp0))); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); // #2 seek, request, poll, response subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp1, this.nextRecords, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); subscriptions.pause(tp0); consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertEquals(1, fetchedRecords.size(), "Should return completed fetch for unpaused partitions"); assertTrue(fetcher.hasCompletedFetches(), "Should still contain completed fetches"); assertNotNull(fetchedRecords.get(tp1)); @@ -1439,13 +1421,13 @@ public class FetcherTest { // #1 seek, request, poll, response subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp0))); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); // #2 seek, request, poll, response subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp1, this.nextRecords, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp1, nextRecords, Errors.NONE, 100L, 0)); subscriptions.pause(tp0); subscriptions.pause(tp1); @@ -1461,7 +1443,7 @@ public class FetcherTest { public void testPartialFetchWithPausedPartitions() { // this test sends creates a completed fetch with 3 records and a max poll of 2 records to assert // that a fetch that must be returned over at least 2 polls can be cached successfully when its partition is - // paused, then returned successfully after its been resumed again later + // paused, then returned successfully after it has been resumed again later buildFetcher(2); Map>> fetchedRecords; @@ -1470,10 +1452,10 @@ public class FetcherTest { subscriptions.seek(tp0, 1); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertEquals(2, fetchedRecords.get(tp0).size(), "Should return 2 records from fetch with 3 records"); assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); @@ -1481,8 +1463,6 @@ public class FetcherTest { subscriptions.pause(tp0); consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); - assertEmptyFetch("Should not return records or advance position for paused partitions"); assertTrue(fetcher.hasCompletedFetches(), "Should have 1 entry in completed fetches"); assertFalse(fetcher.hasAvailableFetches(), "Should not have any available (non-paused) completed fetches"); @@ -1491,7 +1471,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertEquals(1, fetchedRecords.get(tp0).size(), "Should return last remaining record"); assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); @@ -1505,7 +1485,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); subscriptions.pause(tp0); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); subscriptions.seek(tp0, 3); subscriptions.resume(tp0); @@ -1518,97 +1498,51 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches(), "Should have no completed fetches"); } - @Test - public void testFetchNotLeaderOrFollower() { + @ParameterizedTest + @MethodSource("handleFetchResponseErrorSupplier") + public void testHandleFetchResponseError(Errors error, + long highWatermark, + boolean hasTopLevelError, + boolean shouldRequestMetadataUpdate) { buildFetcher(); assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 0); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0)); - consumerClient.poll(time.timer(0)); - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); - } - @Test - public void testFetchUnknownTopicOrPartition() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); + final FetchResponse fetchResponse; - assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.UNKNOWN_TOPIC_OR_PARTITION, 100L, 0)); - consumerClient.poll(time.timer(0)); - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); - } + if (hasTopLevelError) + fetchResponse = fetchResponseWithTopLevelError(tidp0, error, 0); + else + fetchResponse = fullFetchResponse(tidp0, records, error, highWatermark, 0); - @Test - public void testFetchUnknownTopicId() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); - - assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.UNKNOWN_TOPIC_ID, -1L, 0)); + client.prepareResponse(fetchResponse); consumerClient.poll(time.timer(0)); - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); - } - @Test - public void testFetchSessionIdError() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); - - assertEquals(1, sendFetches()); - client.prepareResponse(fetchResponseWithTopLevelError(tidp0, Errors.FETCH_SESSION_TOPIC_ID_ERROR, 0)); - consumerClient.poll(time.timer(0)); assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); - } - @Test - public void testFetchInconsistentTopicId() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); + long timeToNextUpdate = metadata.timeToNextUpdate(time.milliseconds()); - assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.INCONSISTENT_TOPIC_ID, -1L, 0)); - consumerClient.poll(time.timer(0)); - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + if (shouldRequestMetadataUpdate) + assertEquals(0L, timeToNextUpdate, "Should have requested metadata update"); + else + assertNotEquals(0L, timeToNextUpdate, "Should not have requested metadata update"); } - @Test - public void testFetchFencedLeaderEpoch() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); - - assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.FENCED_LEADER_EPOCH, 100L, 0)); - consumerClient.poll(time.timer(0)); - - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()), "Should have requested metadata update"); - } - - @Test - public void testFetchUnknownLeaderEpoch() { - buildFetcher(); - assignFromUser(singleton(tp0)); - subscriptions.seek(tp0, 0); - - assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.UNKNOWN_LEADER_EPOCH, 100L, 0)); - consumerClient.poll(time.timer(0)); - - assertEmptyFetch("Should not return records or advance position on fetch error"); - assertNotEquals(0L, metadata.timeToNextUpdate(time.milliseconds()), "Should not have requested metadata update"); + /** + * Supplies parameters to {@link #testHandleFetchResponseError(Errors, long, boolean, boolean)}. + */ + private static Stream handleFetchResponseErrorSupplier() { + return Stream.of( + Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER, 100L, false, true), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, 100L, false, true), + Arguments.of(Errors.UNKNOWN_TOPIC_ID, -1L, false, true), + Arguments.of(Errors.FETCH_SESSION_TOPIC_ID_ERROR, -1L, true, true), + Arguments.of(Errors.INCONSISTENT_TOPIC_ID, -1L, false, true), + Arguments.of(Errors.FENCED_LEADER_EPOCH, 100L, false, true), + Arguments.of(Errors.UNKNOWN_LEADER_EPOCH, 100L, false, false) + ); } @Test @@ -1636,8 +1570,9 @@ public class FetcherTest { return false; } }; - client.prepareResponse(matcher, fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(matcher, fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.pollNoWakeup(); + assertEquals(0, consumerClient.pendingRequestCount()); } @Test @@ -1647,7 +1582,7 @@ public class FetcherTest { subscriptions.seek(tp0, 0); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); consumerClient.poll(time.timer(0)); assertEmptyFetch("Should not return records or advance position on fetch error"); assertTrue(subscriptions.isOffsetResetNeeded(tp0)); @@ -1664,7 +1599,7 @@ public class FetcherTest { subscriptions.seek(tp0, 0); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); subscriptions.seek(tp0, 1); consumerClient.poll(time.timer(0)); assertEmptyFetch("Should not return records or advance position on fetch error"); @@ -1681,8 +1616,13 @@ public class FetcherTest { subscriptions.seek(tp0, 0); assertTrue(sendFetches() > 0); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); consumerClient.poll(time.timer(0)); + + // The partition is not marked as needing its offset reset because that error handling logic is + // performed during the fetch collection. When we call seek() before we collect the fetch, the + // partition's position is updated (to offset 2) which is different from the offset from which + // we fetched the data (from offset 0). assertFalse(subscriptions.isOffsetResetNeeded(tp0)); subscriptions.seek(tp0, 2); assertEmptyFetch("Should not return records or advance position after seeking to end of topic partition"); @@ -1697,13 +1637,13 @@ public class FetcherTest { subscriptions.seek(tp0, 0); sendFetches(); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, 0)); consumerClient.poll(time.timer(0)); assertFalse(subscriptions.isOffsetResetNeeded(tp0)); for (int i = 0; i < 2; i++) { OffsetOutOfRangeException e = assertThrows(OffsetOutOfRangeException.class, () -> - fetcher.collectFetch()); + collectFetch()); assertEquals(singleton(tp0), e.offsetOutOfRangePartitions().keySet()); assertEquals(0L, e.offsetOutOfRangePartitions().get(tp0).longValue()); } @@ -1752,7 +1692,7 @@ public class FetcherTest { } private void fetchRecordsInto(List> allFetchedRecords) { - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); fetchedRecords.values().forEach(allFetchedRecords::addAll); } @@ -1795,9 +1735,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); List> fetchedRecords = new ArrayList<>(); - Map>> recordsByPartition = fetchedRecords(); - for (List> records : recordsByPartition.values()) - fetchedRecords.addAll(records); + fetchRecordsInto(fetchedRecords); assertEquals(fetchedRecords.size(), subscriptions.position(tp1).offset - 1); assertEquals(4, subscriptions.position(tp1).offset); @@ -1805,9 +1743,7 @@ public class FetcherTest { List oorExceptions = new ArrayList<>(); try { - recordsByPartition = fetchedRecords(); - for (List> records : recordsByPartition.values()) - fetchedRecords.addAll(records); + fetchRecordsInto(fetchedRecords); } catch (OffsetOutOfRangeException oor) { oorExceptions.add(oor); } @@ -1818,9 +1754,7 @@ public class FetcherTest { assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0)); assertEquals(oor.offsetOutOfRangePartitions().size(), 1); - recordsByPartition = fetchedRecords(); - for (List> records : recordsByPartition.values()) - fetchedRecords.addAll(records); + fetchRecordsInto(fetchedRecords); // Should not have received an Exception for tp2. assertEquals(6, subscriptions.position(tp2).offset); @@ -1830,9 +1764,7 @@ public class FetcherTest { List kafkaExceptions = new ArrayList<>(); for (int i = 1; i <= numExceptionsExpected; i++) { try { - recordsByPartition = fetchedRecords(); - for (List> records : recordsByPartition.values()) - fetchedRecords.addAll(records); + fetchRecordsInto(fetchedRecords); } catch (KafkaException e) { kafkaExceptions.add(e); } @@ -1854,10 +1786,10 @@ public class FetcherTest { .setPartitionIndex(tp0.partition()) .setHighWatermark(100) .setRecords(records)); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - assertEquals(2, fetchedRecords().get(tp0).size()); + assertEquals(2, fetchRecords().get(tp0).size()); subscriptions.assignFromUser(mkSet(tp0, tp1)); subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); @@ -1870,7 +1802,7 @@ public class FetcherTest { .setHighWatermark(100)); client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); consumerClient.poll(time.timer(0)); - assertEquals(1, fetchedRecords().get(tp0).size()); + assertEquals(1, fetchRecords().get(tp0).size()); subscriptions.seek(tp1, 10); // Should not throw OffsetOutOfRangeException after the seek @@ -1885,11 +1817,11 @@ public class FetcherTest { subscriptions.seek(tp0, 0); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); consumerClient.poll(time.timer(0)); assertEmptyFetch("Should not return records or advance position on disconnect"); - // disconnects should have no affect on subscription state + // disconnects should have no effect on subscription state assertFalse(subscriptions.isOffsetResetNeeded(tp0)); assertTrue(subscriptions.isFetchable(tp0)); assertEquals(0, subscriptions.position(tp0).offset); @@ -2118,7 +2050,7 @@ public class FetcherTest { client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, fetchPartitionData)); consumerClient.poll(time.timer(0)); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertEquals(3, fetchedRecords.get(tp1).size()); assertEquals(3, fetchedRecords.get(tp2).size()); @@ -2130,7 +2062,7 @@ public class FetcherTest { } @Test - public void testFetchResponseMetricsPartialResponse() { + public void testFetchResponseMetricsWithSkippedOffset() { buildFetcher(); assignFromUser(singleton(tp0)); @@ -2189,7 +2121,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); consumerClient.poll(time.timer(0)); - fetcher.collectFetch(); + collectFetch(); int expectedBytes = 0; for (Record record : records.records()) @@ -2235,7 +2167,7 @@ public class FetcherTest { client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); consumerClient.poll(time.timer(0)); - fetcher.collectFetch(); + collectFetch(); // we should have ignored the record at the wrong offset int expectedBytes = 0; @@ -2256,10 +2188,10 @@ public class FetcherTest { assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 0); assertEquals(1, sendFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); // Verify that all metrics except metrics-count have registered templates @@ -2282,7 +2214,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); client.prepareResponse(fullFetchResponse(tp, records, error, hw, lastStableOffset, throttleTime)); consumerClient.poll(time.timer(0)); - return fetchedRecords(); + return fetchRecords(); } private Map>> fetchRecords( @@ -2290,7 +2222,7 @@ public class FetcherTest { assertEquals(1, sendFetches()); client.prepareResponse(fetchResponse(tp, records, error, hw, lastStableOffset, logStartOffset, throttleTime)); consumerClient.poll(time.timer(0)); - return fetchedRecords(); + return fetchRecords(); } @Test @@ -2359,7 +2291,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); assertEquals(fetchedRecords.get(tp0).size(), 2); } @@ -2429,7 +2361,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); // There are only 3 committed records List> fetchedConsumerRecords = fetchedRecords.get(tp0); @@ -2477,16 +2409,16 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); assertEquals(fetchedRecords.get(tp0).size(), 2); List> fetchedConsumerRecords = fetchedRecords.get(tp0); - Set committedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2")); + Set expectedCommittedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2")); Set actuallyCommittedKeys = new HashSet<>(); for (ConsumerRecord consumerRecord : fetchedConsumerRecords) { actuallyCommittedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8)); } - assertEquals(actuallyCommittedKeys, committedKeys); + assertEquals(expectedCommittedKeys, actuallyCommittedKeys); } @Test @@ -2522,7 +2454,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> allFetchedRecords = fetchedRecords(); + Map>> allFetchedRecords = fetchRecords(); assertTrue(allFetchedRecords.containsKey(tp0)); List> fetchedRecords = allFetchedRecords.get(tp0); assertEquals(3, fetchedRecords.size()); @@ -2561,7 +2493,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> allFetchedRecords = fetchedRecords(); + Map>> allFetchedRecords = fetchRecords(); assertTrue(allFetchedRecords.containsKey(tp0)); List> fetchedRecords = allFetchedRecords.get(tp0); assertEquals(3, fetchedRecords.size()); @@ -2662,7 +2594,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> allFetchedRecords = fetchedRecords(); + Map>> allFetchedRecords = fetchRecords(); assertTrue(allFetchedRecords.containsKey(tp0)); List> fetchedRecords = allFetchedRecords.get(tp0); assertEquals(5, fetchedRecords.size()); @@ -2670,7 +2602,7 @@ public class FetcherTest { } @Test - public void testReturnAbortedTransactionsinUncommittedMode() { + public void testReturnAbortedTransactionsInUncommittedMode() { buildFetcher(OffsetResetStrategy.EARLIEST, new ByteArrayDeserializer(), new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED); ByteBuffer buffer = ByteBuffer.allocate(1024); @@ -2699,7 +2631,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); } @@ -2732,7 +2664,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); // Ensure that we don't return any of the aborted records, but yet advance the consumer position. assertFalse(fetchedRecords.containsKey(tp0)); @@ -2743,7 +2675,6 @@ public class FetcherTest { public void testConsumingViaIncrementalFetchRequests() { buildFetcher(2); - List> records; assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1))); subscriptions.seekValidated(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), metadata.currentLeader(tp0))); subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); @@ -2755,7 +2686,7 @@ public class FetcherTest { .setHighWatermark(2) .setLastStableOffset(2) .setLogStartOffset(0) - .setRecords(this.records)); + .setRecords(records)); partitions1.put(tidp1, new FetchResponseData.PartitionData() .setPartitionIndex(tp1.partition()) .setHighWatermark(100) @@ -2767,9 +2698,9 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches()); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertFalse(fetchedRecords.containsKey(tp1)); - records = fetchedRecords.get(tp0); + List> records = fetchedRecords.get(tp0); assertEquals(2, records.size()); assertEquals(3L, subscriptions.position(tp0).offset); assertEquals(1L, subscriptions.position(tp1).offset); @@ -2778,7 +2709,7 @@ public class FetcherTest { // There is still a buffered record. assertEquals(0, sendFetches()); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertFalse(fetchedRecords.containsKey(tp1)); records = fetchedRecords.get(tp0); assertEquals(1, records.size()); @@ -2791,7 +2722,7 @@ public class FetcherTest { client.prepareResponse(resp2); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.isEmpty()); assertEquals(4L, subscriptions.position(tp0).offset); assertEquals(1L, subscriptions.position(tp1).offset); @@ -2803,12 +2734,12 @@ public class FetcherTest { .setHighWatermark(100) .setLastStableOffset(4) .setLogStartOffset(0) - .setRecords(this.nextRecords)); + .setRecords(nextRecords)); FetchResponse resp3 = FetchResponse.of(Errors.NONE, 0, 123, partitions3); client.prepareResponse(resp3); assertEquals(1, sendFetches()); consumerClient.poll(time.timer(0)); - fetchedRecords = fetchedRecords(); + fetchedRecords = fetchRecords(); assertFalse(fetchedRecords.containsKey(tp1)); records = fetchedRecords.get(tp0); assertEquals(2, records.size()); @@ -2840,7 +2771,8 @@ public class FetcherTest { isolationLevel, apiVersions); - FetchConfig fetchConfig = new FetchConfig<>( + Deserializers deserializers = new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()); + FetchConfig fetchConfig = new FetchConfig( minBytes, maxBytes, maxWaitMs, @@ -2848,7 +2780,6 @@ public class FetcherTest { 2 * numPartitions, true, // check crcs CommonClientConfigs.DEFAULT_CLIENT_RACK, - new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()), isolationLevel); fetcher = new Fetcher( logContext, @@ -2856,6 +2787,7 @@ public class FetcherTest { metadata, subscriptions, fetchConfig, + deserializers, metricsManager, time) { @Override @@ -2947,7 +2879,7 @@ public class FetcherTest { } } if (fetcher.hasCompletedFetches()) { - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); if (!fetchedRecords.isEmpty()) { fetchesRemaining.decrementAndGet(); fetchedRecords.forEach((tp, records) -> { @@ -3009,7 +2941,7 @@ public class FetcherTest { } } if (fetcher.hasCompletedFetches()) { - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); if (!fetchedRecords.isEmpty()) { fetchesRemaining.decrementAndGet(); List> records = fetchedRecords.get(tp0); @@ -3018,7 +2950,7 @@ public class FetcherTest { assertEquals(nextFetchOffset + 1, records.get(1).offset()); nextFetchOffset += 2; } - assertTrue(fetchedRecords().isEmpty()); + assertTrue(fetchRecords().isEmpty()); } } assertEquals(0, future.get()); @@ -3061,7 +2993,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> fetchedRecords = fetchedRecords(); + Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); assertEquals(fetchedRecords.get(tp0).size(), 2); } @@ -3145,7 +3077,7 @@ public class FetcherTest { consumerClient.pollNoWakeup(); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); assertEquals(subscriptions.position(tp0).offset, 3L); @@ -3223,7 +3155,7 @@ public class FetcherTest { consumerClient.pollNoWakeup(); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); assertEquals(subscriptions.position(tp0).offset, 3L); @@ -3239,7 +3171,7 @@ public class FetcherTest { client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); subscriptions.seek(tp0, 0); - // Node preferred replica before first fetch response + // Take note of the preferred replica before the first fetch response Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(-1, selected.id()); @@ -3247,12 +3179,12 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches()); // Set preferred read replica to node=1 - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - Map>> partitionRecords = fetchedRecords(); + Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); // Verify @@ -3264,11 +3196,11 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches()); // Set preferred read replica to node=2, which isn't in our metadata, should revert to leader - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(2))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(-1, selected.id()); } @@ -3284,11 +3216,11 @@ public class FetcherTest { assertEquals(1, sendFetches()); // Set preferred read replica to node=1 - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Verify Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); @@ -3297,11 +3229,11 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches()); // Disconnect - preferred read replica should be cleared. - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); consumerClient.poll(time.timer(0)); assertFalse(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(-1, selected.id()); } @@ -3317,11 +3249,11 @@ public class FetcherTest { assertEquals(1, sendFetches()); // Set preferred read replica to node=1 - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Verify Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); @@ -3330,13 +3262,13 @@ public class FetcherTest { assertFalse(fetcher.hasCompletedFetches()); // Disconnect and remove tp0 from assignment - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, 0), true); subscriptions.assignFromUser(emptySet()); // Preferred read replica should not be cleared consumerClient.poll(time.timer(0)); assertFalse(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(-1, selected.id()); } @@ -3352,11 +3284,11 @@ public class FetcherTest { assertEquals(1, sendFetches()); // Set preferred read replica to node=1 - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); // Verify Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); @@ -3371,7 +3303,7 @@ public class FetcherTest { consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(-1, selected.id()); } @@ -3389,26 +3321,26 @@ public class FetcherTest { assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.NONE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(selected.id(), 1); - // Return an error, should unset the preferred read replica assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); - client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.OFFSET_OUT_OF_RANGE, 100L, + // Return an error, should unset the preferred read replica + client.prepareResponse(fullFetchResponse(tidp0, records, Errors.OFFSET_OUT_OF_RANGE, 100L, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.empty())); consumerClient.poll(time.timer(0)); assertTrue(fetcher.hasCompletedFetches()); - fetchedRecords(); + fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); assertEquals(selected.id(), -1); @@ -3422,7 +3354,7 @@ public class FetcherTest { sendFetches(); client.prepareResponse(fullFetchResponse(tidp0, buildRecords(1L, 1, 1), Errors.NONE, 100L, 0)); consumerClient.poll(time.timer(0)); - fetchedRecords(); + fetchRecords(); Metadata.LeaderAndEpoch leaderAndEpoch = subscriptions.position(tp0).currentLeader; assertTrue(leaderAndEpoch.leader.isPresent()); @@ -3463,7 +3395,7 @@ public class FetcherTest { assertTrue(fetcher.hasCompletedFetches()); // Trigger the exception. - assertThrows(KafkaException.class, this::fetchedRecords); + assertThrows(KafkaException.class, this::fetchRecords); } private OffsetsForLeaderEpochResponse prepareOffsetsForLeaderEpochResponse( @@ -3577,14 +3509,14 @@ public class FetcherTest { assertTrue(fetch.isEmpty(), reason); } - private Map>> fetchedRecords() { + private Map>> fetchRecords() { Fetch fetch = collectFetch(); return fetch.records(); } @SuppressWarnings("unchecked") private Fetch collectFetch() { - return (Fetch) fetcher.collectFetch(); + return (Fetch) fetcher.collectFetch(); } private void buildFetcher(int maxPollRecords) { @@ -3642,7 +3574,7 @@ public class FetcherTest { SubscriptionState subscriptionState, LogContext logContext) { buildDependencies(metricConfig, metadataExpireMs, subscriptionState, logContext); - FetchConfig fetchConfig = new FetchConfig<>( + FetchConfig fetchConfig = new FetchConfig( minBytes, maxBytes, maxWaitMs, @@ -3650,7 +3582,6 @@ public class FetcherTest { maxPollRecords, true, // check crc CommonClientConfigs.DEFAULT_CLIENT_RACK, - new Deserializers<>(keyDeserializer, valueDeserializer), isolationLevel); fetcher = spy(new Fetcher<>( logContext, @@ -3658,6 +3589,7 @@ public class FetcherTest { metadata, subscriptionState, fetchConfig, + new Deserializers<>(keyDeserializer, valueDeserializer), metricsManager, time)); offsetFetcher = new OffsetFetcher(logContext, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 7d635ddc284..480a6242e40 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.TimeoutException; @@ -28,11 +27,9 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -46,80 +43,77 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Optional; -import java.util.Properties; -import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_MAX_POLL_INTERVAL_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class HeartbeatRequestManagerTest { - private static final int HEARTBEAT_INTERVAL_MS = 1000; - private static final long RETRY_BACKOFF_MAX_MS = 3000; - private static final long RETRY_BACKOFF_MS = 100; - private static final String GROUP_INSTANCE_ID = "group-instance-id"; - private static final String GROUP_ID = "group-id"; - + private ConsumerTestBuilder testBuilder; private Time time; - private LogContext logContext; private CoordinatorRequestManager coordinatorRequestManager; - private SubscriptionState subscriptionState; + private SubscriptionState subscriptions; private HeartbeatRequestManager heartbeatRequestManager; private MembershipManager membershipManager; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; - private ConsumerConfig config; - - private String memberId = "member-id"; - private int memberEpoch = 1; - private ErrorEventHandler errorEventHandler; + private final String memberId = "member-id"; + private final int memberEpoch = 1; + private BackgroundEventHandler backgroundEventHandler; @BeforeEach public void setUp() { - time = new MockTime(); - logContext = new LogContext(); - config = new ConsumerConfig(createConsumerConfig()); - coordinatorRequestManager = mock(CoordinatorRequestManager.class); + setUp(ConsumerTestBuilder.createDefaultGroupInformation()); + } + + private void setUp(Optional groupInfo) { + testBuilder = new ConsumerTestBuilder(groupInfo); + time = testBuilder.time; + coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); + heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new); + heartbeatRequestState = testBuilder.heartbeatRequestState.orElseThrow(IllegalStateException::new); + backgroundEventHandler = testBuilder.backgroundEventHandler; + subscriptions = testBuilder.subscriptions; + membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - subscriptionState = mock(SubscriptionState.class); - membershipManager = spy(new MembershipManagerImpl(GROUP_ID, logContext)); - heartbeatRequestState = mock(HeartbeatRequestManager.HeartbeatRequestState.class); - errorEventHandler = mock(ErrorEventHandler.class); - heartbeatRequestManager = createManager(); } - private Properties createConsumerConfig() { - Properties properties = new Properties(); - properties.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(RETRY_BACKOFF_MS_CONFIG, "100"); - return properties; + private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { + cleanup(); + + ConsumerTestBuilder.GroupInformation gi = new ConsumerTestBuilder.GroupInformation( + new GroupState(DEFAULT_GROUP_ID, groupInstanceId), + 0, + 0.0 + ); + + setUp(Optional.of(gi)); + } + + @AfterEach + public void cleanup() { + if (testBuilder != null) { + testBuilder.close(); + } } @Test public void testHeartbeatOnStartup() { // The initial heartbeatInterval is set to 0 - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - 0, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS, - 0); - heartbeatRequestManager = createManager(); + resetWithZeroHeartbeatInterval(Optional.empty()); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -131,6 +125,9 @@ public class HeartbeatRequestManagerTest { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testSendHeartbeatOnMemberState(final boolean shouldSendHeartbeat) { + // The initial heartbeatInterval is set to 0 + resetWithZeroHeartbeatInterval(Optional.empty()); + // Mocking notInGroup when(membershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); @@ -150,21 +147,13 @@ public class HeartbeatRequestManagerTest { @ParameterizedTest @MethodSource("stateProvider") public void testTimerNotDue(final MemberState state) { - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - HEARTBEAT_INTERVAL_MS, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS); - heartbeatRequestManager = createManager(); - when(membershipManager.state()).thenReturn(state); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); if (membershipManager.shouldSendHeartbeat()) { - assertEquals(HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); } else { assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); } @@ -172,14 +161,8 @@ public class HeartbeatRequestManagerTest { @Test public void testNetworkTimeout() { - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - 0, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS, - 0); - heartbeatRequestManager = createManager(); + // The initial heartbeatInterval is set to 0 + resetWithZeroHeartbeatInterval(Optional.empty()); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSendHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -188,7 +171,7 @@ public class HeartbeatRequestManagerTest { result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout")); // Assure the manager will backoff on timeout - time.sleep(RETRY_BACKOFF_MS - 1); + time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); @@ -199,21 +182,16 @@ public class HeartbeatRequestManagerTest { @Test public void testFailureOnFatalException() { - heartbeatRequestState = spy(new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - 0, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS, - 0)); - heartbeatRequestManager = createManager(); + // The initial heartbeatInterval is set to 0 + resetWithZeroHeartbeatInterval(Optional.empty()); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSendHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); result.unsentRequests.get(0).future().completeExceptionally(new KafkaException("fatal")); verify(membershipManager).transitionToFailed(); - verify(errorEventHandler).handle(any()); + verify(backgroundEventHandler).add(any()); } @Test @@ -228,22 +206,12 @@ public class HeartbeatRequestManagerTest { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest(final short version) { + // The initial heartbeatInterval is set to 0, but we're testing + resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + List subscribedTopics = Collections.singletonList("topic"); - subscriptionState = new SubscriptionState(logContext, OffsetResetStrategy.NONE); - subscriptionState.subscribe(new HashSet<>(subscribedTopics), new NoOpConsumerRebalanceListener()); - - Properties prop = createConsumerConfig(); - prop.setProperty(MAX_POLL_INTERVAL_MS_CONFIG, "10000"); - config = new ConsumerConfig(prop); - membershipManager = new MembershipManagerImpl(GROUP_ID, GROUP_INSTANCE_ID, null, logContext); - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - 0, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS, - 0); - heartbeatRequestManager = createManager(); + subscriptions.subscribe(new HashSet<>(subscribedTopics), new NoOpConsumerRebalanceListener()); + // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() @@ -259,12 +227,12 @@ public class HeartbeatRequestManagerTest { ConsumerGroupHeartbeatRequest heartbeatRequest = (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); - assertEquals(GROUP_ID, heartbeatRequest.data().groupId()); + assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); assertEquals(memberId, heartbeatRequest.data().memberId()); assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch()); - assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs()); + assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, heartbeatRequest.data().rebalanceTimeoutMs()); assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames()); - assertEquals(GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId()); + assertEquals(DEFAULT_GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId()); // TODO: Test pattern subscription and user provided assignor selection. assertNull(heartbeatRequest.data().serverAssignor()); assertNull(heartbeatRequest.data().subscribedTopicRegex()); @@ -273,25 +241,16 @@ public class HeartbeatRequestManagerTest { @ParameterizedTest @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - HEARTBEAT_INTERVAL_MS, - RETRY_BACKOFF_MS, - RETRY_BACKOFF_MAX_MS, - 0); - heartbeatRequestManager = createManager(); - // Sending first heartbeat w/o assignment to set the state to STABLE ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(HEARTBEAT_INTERVAL_MS) + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) .setMemberId(memberId) .setMemberEpoch(memberEpoch)); membershipManager.updateState(rs1.data()); assertEquals(MemberState.STABLE, membershipManager.state()); // Handling errors on the second heartbeat - time.sleep(HEARTBEAT_INTERVAL_MS); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -304,19 +263,19 @@ public class HeartbeatRequestManagerTest { switch (error) { case NONE: - verify(errorEventHandler, never()).handle(any()); + verify(backgroundEventHandler, never()).add(any()); verify(membershipManager, times(2)).updateState(mockResponse.data()); - assertEquals(HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); break; case COORDINATOR_LOAD_IN_PROGRESS: - verify(errorEventHandler, never()).handle(any()); - assertEquals(RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); + verify(backgroundEventHandler, never()).add(any()); + assertEquals(DEFAULT_RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); break; case COORDINATOR_NOT_AVAILABLE: case NOT_COORDINATOR: - verify(errorEventHandler, never()).handle(any()); + verify(backgroundEventHandler, never()).add(any()); verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong()); assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); break; @@ -326,7 +285,7 @@ public class HeartbeatRequestManagerTest { // The memberStateManager should have stopped heartbeat at this point ensureFatalError(); } else { - verify(errorEventHandler, never()).handle(any()); + verify(backgroundEventHandler, never()).add(any()); assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); } break; @@ -335,12 +294,12 @@ public class HeartbeatRequestManagerTest { private void ensureFatalError() { verify(membershipManager).transitionToFailed(); - verify(errorEventHandler).handle(any()); + verify(backgroundEventHandler).add(any()); ensureHeartbeatStopped(); } private void ensureHeartbeatStopped() { - time.sleep(HEARTBEAT_INTERVAL_MS); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(MemberState.FAILED, membershipManager.state()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); @@ -377,7 +336,7 @@ public class HeartbeatRequestManagerTest { final Errors error) { ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData() .setErrorCode(error.code()) - .setHeartbeatIntervalMs(HEARTBEAT_INTERVAL_MS) + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) .setMemberId(memberId) .setMemberEpoch(memberEpoch); if (error != Errors.NONE) { @@ -395,16 +354,4 @@ public class HeartbeatRequestManagerTest { null, response); } - - private HeartbeatRequestManager createManager() { - return new HeartbeatRequestManager( - logContext, - time, - config, - coordinatorRequestManager, - subscriptionState, - membershipManager, - heartbeatRequestState, - errorEventHandler); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java index 4959a1c7b77..a118b20297a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java @@ -564,7 +564,7 @@ public class OffsetFetcherTest { } @Test - public void testRestOffsetsAuthorizationFailure() { + public void testResetOffsetsAuthorizationFailure() { buildFetcher(); assignFromUser(singleton(tp0)); subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); @@ -1246,7 +1246,7 @@ public class OffsetFetcherTest { buildFetcher(metricConfig, isolationLevel, metadataExpireMs, subscriptionState, logContext); FetchMetricsRegistry metricsRegistry = new FetchMetricsRegistry(metricConfig.tags().keySet(), "consumertest-group"); - FetchConfig fetchConfig = new FetchConfig<>( + FetchConfig fetchConfig = new FetchConfig( minBytes, maxBytes, maxWaitMs, @@ -1254,7 +1254,6 @@ public class OffsetFetcherTest { maxPollRecords, true, // check crc CommonClientConfigs.DEFAULT_CLIENT_RACK, - new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()), isolationLevel); Fetcher fetcher = new Fetcher<>( logContext, @@ -1262,6 +1261,7 @@ public class OffsetFetcherTest { metadata, subscriptions, fetchConfig, + new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()), new FetchMetricsManager(metrics, metricsRegistry), time); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java index e6f7b91a8da..5ca034d6360 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java @@ -22,6 +22,9 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.IsolationLevel; @@ -57,15 +60,20 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -84,6 +92,7 @@ public class OffsetsRequestManagerTest { private SubscriptionState subscriptionState; private MockTime time; private ApiVersions apiVersions; + private BlockingQueue backgroundEventQueue; private static final String TEST_TOPIC = "t1"; private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1); private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2); @@ -95,13 +104,25 @@ public class OffsetsRequestManagerTest { @BeforeEach public void setup() { + LogContext logContext = new LogContext(); + backgroundEventQueue = new LinkedBlockingQueue<>(); + BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue); metadata = mock(ConsumerMetadata.class); subscriptionState = mock(SubscriptionState.class); - this.time = new MockTime(0); + time = new MockTime(0); apiVersions = mock(ApiVersions.class); - requestManager = new OffsetsRequestManager(subscriptionState, metadata, - DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, REQUEST_TIMEOUT_MS, - apiVersions, mock(NetworkClientDelegate.class), new LogContext()); + requestManager = new OffsetsRequestManager( + subscriptionState, + metadata, + DEFAULT_ISOLATION_LEVEL, + time, + RETRY_BACKOFF_MS, + REQUEST_TIMEOUT_MS, + apiVersions, + mock(NetworkClientDelegate.class), + backgroundEventHandler, + logContext + ); } @Test @@ -500,7 +521,7 @@ public class OffsetsRequestManagerTest { } @Test - public void testResetPositionsThrowsPreviousException() { + public void testResetOffsetsAuthorizationFailure() { when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1)); when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST); mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); @@ -510,8 +531,9 @@ public class OffsetsRequestManagerTest { // Reset positions response with TopicAuthorizationException NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); + Errors topicAuthorizationFailedError = Errors.TOPIC_AUTHORIZATION_FAILED; ClientResponse clientResponse = buildClientResponseWithErrors( - unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED)); + unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError)); clientResponse.onComplete(); assertTrue(unsentRequest.future().isDone()); @@ -520,11 +542,23 @@ public class OffsetsRequestManagerTest { verify(subscriptionState).requestFailed(any(), anyLong()); verify(metadata).requestUpdate(false); - // Following resetPositions should raise the previous exception without performing any - // request - assertThrows(TopicAuthorizationException.class, - () -> requestManager.resetPositionsIfNeeded()); + // Following resetPositions should enqueue the previous exception in the background event queue + // without performing any request + assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded()); assertEquals(0, requestManager.requestsToSend()); + + // Check that the event was enqueued during resetPositionsIfNeeded + assertEquals(1, backgroundEventQueue.size()); + BackgroundEvent event = backgroundEventQueue.poll(); + assertNotNull(event); + + // Check that the event itself is of the expected type + assertInstanceOf(ErrorBackgroundEvent.class, event); + ErrorBackgroundEvent errorEvent = (ErrorBackgroundEvent) event; + assertNotNull(errorEvent.error()); + + // Check that the error held in the event is of the expected type + assertInstanceOf(topicAuthorizationFailedError.exception().getClass(), errorEvent.error()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java index e5f9c27a28f..d67f509d905 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java @@ -16,15 +16,12 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.EventHandler; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; @@ -35,13 +32,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.ListOffsetsRequest; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -58,20 +49,15 @@ import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; import static java.util.Collections.singleton; -import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -80,56 +66,49 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class PrototypeAsyncConsumerTest { private PrototypeAsyncConsumer consumer; - private final Map consumerProps = new HashMap<>(); - - private final Time time = new MockTime(); - private LogContext logContext; - private SubscriptionState subscriptions; - private ConsumerMetadata metadata; - private EventHandler eventHandler; - private Metrics metrics; - - private String groupId = "group.id"; - private ConsumerConfig config; + private ConsumerTestBuilder.PrototypeAsyncConsumerTestBuilder testBuilder; + private ApplicationEventHandler applicationEventHandler; @BeforeEach public void setup() { - injectConsumerConfigs(); - this.config = new ConsumerConfig(consumerProps); - this.logContext = new LogContext(); - this.subscriptions = mock(SubscriptionState.class); - this.metadata = mock(ConsumerMetadata.class); - final DefaultBackgroundThread bt = mock(DefaultBackgroundThread.class); - final BlockingQueue aq = new LinkedBlockingQueue<>(); - final BlockingQueue bq = new LinkedBlockingQueue<>(); - this.eventHandler = spy(new DefaultEventHandler(bt, aq, bq)); - this.metrics = new Metrics(time); + // By default, the consumer is part of a group. + setup(ConsumerTestBuilder.createDefaultGroupInformation()); + } + + private void setup(Optional groupInfo) { + testBuilder = new ConsumerTestBuilder.PrototypeAsyncConsumerTestBuilder(groupInfo); + applicationEventHandler = testBuilder.applicationEventHandler; + consumer = testBuilder.consumer; } @AfterEach public void cleanup() { - if (consumer != null) { - consumer.close(Duration.ZERO); + if (testBuilder != null) { + testBuilder.close(); } } + private void resetWithEmptyGroupId() { + // Create a consumer that is not configured as part of a group. + cleanup(); + setup(Optional.empty()); + } + @Test public void testSuccessfulStartupShutdown() { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertDoesNotThrow(() -> consumer.close()); } @Test public void testInvalidGroupId() { - this.groupId = null; - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + // Create consumer without group id + resetWithEmptyGroupId(); assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>())); } @@ -140,11 +119,10 @@ public class PrototypeAsyncConsumerTest { offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L)); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - PrototypeAsyncConsumer mockedConsumer = spy(newConsumer(time, new StringDeserializer(), new StringDeserializer())); - doReturn(future).when(mockedConsumer).commit(offsets, false); - mockedConsumer.commitAsync(offsets, null); + doReturn(future).when(consumer).commit(offsets, false); + consumer.commitAsync(offsets, null); future.complete(null); - TestUtils.waitForCondition(() -> future.isDone(), + TestUtils.waitForCondition(future::isDone, 2000, "commit future should complete"); @@ -159,11 +137,9 @@ public class PrototypeAsyncConsumerTest { offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L)); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); - PrototypeAsyncConsumer mockedConsumer = spy(consumer); - doReturn(future).when(mockedConsumer).commit(offsets, false); + doReturn(future).when(consumer).commit(offsets, false); OffsetCommitCallback customCallback = mock(OffsetCommitCallback.class); - mockedConsumer.commitAsync(offsets, customCallback); + consumer.commitAsync(offsets, customCallback); future.complete(null); verify(customCallback).onComplete(offsets, null); } @@ -175,9 +151,8 @@ public class PrototypeAsyncConsumerTest { committedFuture.complete(offsets); try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertDoesNotThrow(() -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); - verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); } } @@ -188,9 +163,8 @@ public class PrototypeAsyncConsumerTest { committedFuture.completeExceptionally(new KafkaException("Test exception")); try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); - verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); } } @@ -230,25 +204,21 @@ public class PrototypeAsyncConsumerTest { @Test public void testAssign() { - this.subscriptions = new SubscriptionState(logContext, OffsetResetStrategy.EARLIEST); - this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); final TopicPartition tp = new TopicPartition("foo", 3); consumer.assign(singleton(tp)); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().contains(tp)); - verify(eventHandler).add(any(AssignmentChangeApplicationEvent.class)); - verify(eventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); + verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); } @Test public void testAssignOnNullTopicPartition() { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); } @Test public void testAssignOnEmptyTopicPartition() { - consumer = spy(newConsumer(time, new StringDeserializer(), new StringDeserializer())); consumer.assign(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); @@ -256,76 +226,68 @@ public class PrototypeAsyncConsumerTest { @Test public void testAssignOnNullTopicInPartition() { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); } @Test public void testAssignOnEmptyTopicInPartition() { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); } @Test public void testBeginningOffsetsFailsIfNullPartitions() { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, Duration.ofMillis(1))); } @Test public void testBeginningOffsets() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(eventHandler).addAndGet(any(), any()); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue().offset())); + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); Set partitions = mockTopicPartitionOffset().keySet(); Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); - doThrow(eventProcessingFailure).when(eventHandler).addAndGet(any(), any()); + doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any()); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); - doThrow(new TimeoutException()).when(eventHandler).addAndGet(any(), any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test public void testOffsetsForTimesOnNullPartitions() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, Duration.ofMillis(1))); } @Test public void testOffsetsForTimesFailsOnNegativeTargetTimes() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); assertThrows(IllegalArgumentException.class, () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), @@ -344,15 +306,14 @@ public class PrototypeAsyncConsumerTest { @Test public void testOffsetsForTimes() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(eventHandler).addAndGet(any(), any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -361,7 +322,6 @@ public class PrototypeAsyncConsumerTest { // OffsetAndTimestamp as value. @Test public void testOffsetsForTimesWithZeroTimeout() { - PrototypeAsyncConsumer consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); TopicPartition tp = new TopicPartition("topic1", 0); Map expectedResult = Collections.singletonMap(tp, null); @@ -371,14 +331,12 @@ public class PrototypeAsyncConsumerTest { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(0))); assertEquals(expectedResult, result); - verify(eventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test public void testWakeup_committed() { - consumer = newConsumer(time, new StringDeserializer(), - new StringDeserializer()); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(mockTopicPartitionOffset().keySet())); assertNoPendingWakeup(consumer.wakeupTrigger()); @@ -386,87 +344,80 @@ public class PrototypeAsyncConsumerTest { @Test public void testRefreshCommittedOffsetsSuccess() { - Map committedOffsets = - Collections.singletonMap(new TopicPartition("t1", 1), new OffsetAndMetadata(10L)); - testRefreshCommittedOffsetsSuccess(committedOffsets); + TopicPartition partition = new TopicPartition("t1", 1); + Set partitions = Collections.singleton(partition); + Map committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L)); + testRefreshCommittedOffsetsSuccess(partitions, committedOffsets); } @Test public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() { - testRefreshCommittedOffsetsSuccess(Collections.emptyMap()); + TopicPartition partition = new TopicPartition("t1", 1); + Set partitions = Collections.singleton(partition); + Map committedOffsets = Collections.emptyMap(); + testRefreshCommittedOffsetsSuccess(partitions, committedOffsets); } @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { - // Create consumer with group id to enable committed offset usage - this.groupId = "consumer-group-1"; - testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true); } @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { // Create consumer without group id so committed offsets are not used for updating positions - this.groupId = null; - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); - + resetWithEmptyGroupId(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); - // Uncompleted future that will time out if used CompletableFuture> committedFuture = new CompletableFuture<>(); - when(subscriptions.initializingPartitions()).thenReturn(Collections.singleton(new TopicPartition("t1", 1))); - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { + consumer.assign(singleton(new TopicPartition("t1", 1))); + try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { // Poll with 0 timeout to run a single iteration of the poll loop consumer.poll(Duration.ofMillis(0)); - verify(eventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); if (committedOffsetsEnabled) { // Verify there was an OffsetFetch event and no ResetPositions event - verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); - verify(eventHandler, + verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); + verify(applicationEventHandler, never()).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } else { // Verify there was not any OffsetFetch event but there should be a ResetPositions - verify(eventHandler, + verify(applicationEventHandler, never()).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); - verify(eventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } } } - private void testRefreshCommittedOffsetsSuccess(Map committedOffsets) { + private void testRefreshCommittedOffsetsSuccess(Set partitions, + Map committedOffsets) { CompletableFuture> committedFuture = new CompletableFuture<>(); committedFuture.complete(committedOffsets); - - // Create consumer with group id to enable committed offset usage - this.groupId = "consumer-group-1"; - consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + consumer.assign(partitions); try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - when(subscriptions.initializingPartitions()).thenReturn(committedOffsets.keySet()); - // Poll with 0 timeout to run a single iteration of the poll loop consumer.poll(Duration.ofMillis(0)); - verify(eventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); - verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); - verify(eventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class)); } } private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) { - assertTrue(wakeupTrigger.getPendingTask() == null); + assertNull(wakeupTrigger.getPendingTask()); } - private Map mockTopicPartitionOffset() { + private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap topicPartitionOffsets = new HashMap<>(); @@ -475,7 +426,7 @@ public class PrototypeAsyncConsumerTest { return topicPartitionOffsets; } - private Map mockOffsetAndTimestamp() { + private HashMap mockOffsetAndTimestamp() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap offsetAndTimestamp = new HashMap<>(); @@ -484,7 +435,7 @@ public class PrototypeAsyncConsumerTest { return offsetAndTimestamp; } - private Map mockTimestampToSearch() { + private HashMap mockTimestampToSearch() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap timestampToSearch = new HashMap<>(); @@ -492,30 +443,5 @@ public class PrototypeAsyncConsumerTest { timestampToSearch.put(t1, 2L); return timestampToSearch; } - - private void injectConsumerConfigs() { - consumerProps.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - consumerProps.put(DEFAULT_API_TIMEOUT_MS_CONFIG, "60000"); - consumerProps.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - consumerProps.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - } - - private PrototypeAsyncConsumer newConsumer(final Time time, - final Deserializer keyDeserializer, - final Deserializer valueDeserializer) { - consumerProps.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); - consumerProps.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); - - return new PrototypeAsyncConsumer<>( - time, - logContext, - config, - subscriptions, - metadata, - eventHandler, - metrics, - Optional.ofNullable(this.groupId), - config.getInt(DEFAULT_API_TIMEOUT_MS_CONFIG)); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java new file mode 100644 index 00000000000..0670b8bdb7c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandlerTest.java @@ -0,0 +1,141 @@ +/* + * 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.events; + +import org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder; +import org.apache.kafka.common.KafkaException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.BlockingQueue; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class BackgroundEventHandlerTest { + + private ConsumerTestBuilder testBuilder; + private BlockingQueue backgroundEventQueue; + private BackgroundEventHandler backgroundEventHandler; + private BackgroundEventProcessor backgroundEventProcessor; + + @BeforeEach + public void setup() { + testBuilder = new ConsumerTestBuilder(); + backgroundEventQueue = testBuilder.backgroundEventQueue; + backgroundEventHandler = testBuilder.backgroundEventHandler; + backgroundEventProcessor = testBuilder.backgroundEventProcessor; + } + + @AfterEach + public void tearDown() { + if (testBuilder != null) + testBuilder.close(); + } + + @Test + public void testNoEvents() { + assertTrue(backgroundEventQueue.isEmpty()); + backgroundEventProcessor.process((event, error) -> { }); + assertTrue(backgroundEventQueue.isEmpty()); + } + + @Test + public void testSingleEvent() { + BackgroundEvent event = new ErrorBackgroundEvent(new RuntimeException("A")); + backgroundEventQueue.add(event); + assertPeeked(event); + backgroundEventProcessor.process((e, error) -> { }); + assertTrue(backgroundEventQueue.isEmpty()); + } + + @Test + public void testSingleErrorEvent() { + KafkaException error = new KafkaException("error"); + BackgroundEvent event = new ErrorBackgroundEvent(error); + backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + assertPeeked(event); + assertProcessThrows(error); + } + + @Test + public void testMultipleEvents() { + BackgroundEvent event1 = new ErrorBackgroundEvent(new RuntimeException("A")); + backgroundEventQueue.add(event1); + backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("B"))); + backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("C"))); + + assertPeeked(event1); + backgroundEventProcessor.process((event, error) -> { }); + assertTrue(backgroundEventQueue.isEmpty()); + } + + @Test + public void testMultipleErrorEvents() { + Throwable error1 = new Throwable("error1"); + KafkaException error2 = new KafkaException("error2"); + KafkaException error3 = new KafkaException("error3"); + + backgroundEventHandler.add(new ErrorBackgroundEvent(error1)); + backgroundEventHandler.add(new ErrorBackgroundEvent(error2)); + backgroundEventHandler.add(new ErrorBackgroundEvent(error3)); + + assertProcessThrows(new KafkaException(error1)); + } + + @Test + public void testMixedEventsWithErrorEvents() { + Throwable error1 = new Throwable("error1"); + KafkaException error2 = new KafkaException("error2"); + KafkaException error3 = new KafkaException("error3"); + + RuntimeException errorToCheck = new RuntimeException("A"); + backgroundEventQueue.add(new ErrorBackgroundEvent(errorToCheck)); + backgroundEventHandler.add(new ErrorBackgroundEvent(error1)); + backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("B"))); + backgroundEventHandler.add(new ErrorBackgroundEvent(error2)); + backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("C"))); + backgroundEventHandler.add(new ErrorBackgroundEvent(error3)); + backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("D"))); + + assertProcessThrows(new KafkaException(errorToCheck)); + } + + private void assertPeeked(BackgroundEvent event) { + BackgroundEvent peekEvent = backgroundEventQueue.peek(); + assertNotNull(peekEvent); + assertEquals(event, peekEvent); + } + + private void assertProcessThrows(Throwable error) { + assertFalse(backgroundEventQueue.isEmpty()); + + try { + backgroundEventProcessor.process(); + fail("Should have thrown error: " + error); + } catch (Throwable t) { + assertEquals(error.getClass(), t.getClass()); + assertEquals(error.getMessage(), t.getMessage()); + } + + assertTrue(backgroundEventQueue.isEmpty()); + } +}