[PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-23 Thread via GitHub


kirktrue opened a new pull request, #14406:
URL: https://github.com/apache/kafka/pull/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.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-02 Thread via GitHub


junrao commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1334821243


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.commo

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344756235


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java:
##
@@ -0,0 +1,136 @@
+/*
+ * 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.LinkedList;
+import java.util.List;
+import java.util.Queue;
+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;
+import org.slf4j.Logger;
+
+/**
+ * {@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 Logger log;
+private final ErrorEventHandler errorEventHandler;
+private final NetworkClientDelegate networkClientDelegate;
+
+FetchRequestManager(final LogContext logContext,
+final Time time,
+final ErrorEventHandler errorEventHandler,
+final ConsumerMetadata metadata,
+final SubscriptionState subscriptions,
+final FetchConfig fetchConfig,
+final FetchMetricsManager metricsManager,
+final NetworkClientDelegate networkClientDelegate) {
+super(logContext, metadata, subscriptions, fetchConfig, 
metricsManager, time);
+this.log = logContext.logger(FetchRequestManager.class);
+this.errorEventHandler = errorEventHandler;
+this.networkClientDelegate = networkClientDelegate;
+}
+
+@Override
+protected boolean isUnavailable(Node node) {
+return networkClientDelegate.isUnavailable(node);
+}
+
+@Override
+protected void maybeThrowAuthFailure(Node node) {
+networkClientDelegate.maybeThrowAuthFailure(node);
+}
+
+@Override
+public PollResult poll(long currentTimeMs) {
+List requests;
+
+if (!idempotentCloser.isClosed()) {
+// If the fetcher is open (i.e. not closed), we will issue the 
normal fetch requests
+requests = prepareFetchRequests().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, t) -> {
+if (t != null) {
+handleFetchResponse(fetchTarget, t);
+log.warn("Attempt to fetch data from node {} failed 
due to fatal exception", fetchTarget, t);
+errorEventHandler.handle(t);
+} else {
+handleFetchResponse(fetchTarget, data, clientResponse);
+}
+};
+
+return new UnsentRequest(request, fetchTarget, 
responseHandler);
+}).collect(Collectors.toList());
+} else {
+requests = 
prepareCloseFetchSessionRequests().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, t) -> {
+if (t != null) {
+

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344758105


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344762836


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##
@@ -106,7 +127,62 @@ public void onFailure(RuntimeException e) {
 return fetchRequestMap.size();
 }
 
+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);

Review Comment:
   Can you look at the new `DefaultBackgroundThread.runAtClose()` method I 
added? Do we need to update the `CommitRequestManager` to implement the 
`pollOnClose()` API I added?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344780023


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344786987


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344786987


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344795530


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344799794


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344801112


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344822614


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java:
##
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+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.NetworkClient;
+import 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
+import 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest;
+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.requests.FetchRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+/**
+ * {@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 Logger log;
+private final BackgroundEventHandler backgroundEventHandler;
+private final NetworkClientDelegate networkClientDelegate;
+private final List>> futures;
+
+FetchRequestManager(final LogContext logContext,
+final Time time,
+final BackgroundEventHandler backgroundEventHandler,
+final ConsumerMetadata metadata,
+final SubscriptionState subscriptions,
+final FetchConfig fetchConfig,
+final FetchMetricsManager metricsManager,
+final NetworkClientDelegate networkClientDelegate) {
+super(logContext, metadata, subscriptions, fetchConfig, 
metricsManager, time);
+this.log = logContext.logger(FetchRequestManager.class);
+this.backgroundEventHandler = backgroundEventHandler;
+this.networkClientDelegate = networkClientDelegate;
+this.futures = new ArrayList<>();
+}
+
+@Override
+protected boolean isUnavailable(Node node) {
+return networkClientDelegate.isUnavailable(node);
+}
+
+@Override
+protected void maybeThrowAuthFailure(Node node) {
+networkClientDelegate.maybeThrowAuthFailure(node);
+}
+
+/**
+ * Adds a new {@link Future future} to the list of futures awaiting 
results. Per the comments on
+ * {@link #forwardResults()}, there is no guarantee that this particular 
future will be provided with
+ * a non-empty result, but it is guaranteed to be completed with a result, 
assuming that it does not time out.
+ *
+ * @param future Future that will be {@link 
CompletableFuture#complete(Object) completed} if not timed out
+ */
+public void requestFetch(CompletableFuture> future) {
+futures.add(future);
+}
+
+@Override
+public PollResult poll(long currentTimeMs) {

Review Comment:
   I added documentation to the `RequestManager` interface with pointers to it 
from the methods in `FetchRequestManager`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344897098


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -37,7 +36,7 @@
  *
  * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
  */
-public class FetchBuffer implements Closeable {
+public class FetchBuffer implements AutoCloseable {

Review Comment:
   Yes, the process is a bit convoluted...
   
   To perform the process of moving the fetched records from the background 
thread to the application thread and then on to the user, 
`PrototypeAsyncConsumer` has these three instance variables:
   
   1. `fetchResults`
   2. `fetchBuffer`
   3. `fetchCollector`
   
   All three of those objects are created in the application thread when the 
`PrototypeAsyncConsumer` is created. `fetchBuffer` and `fetchCollector` are 
only ever referenced by the application thread; `fetchResults`, however, is 
used by **both** threads.
   
   `fetchResults` is referenced in the background thread when it is used in the 
`FetchEvent` callback in the `sendFetches()` method:
   
   ```java
   private void sendFetches() {
   FetchEvent event = new FetchEvent();
   applicationEventHandler.add(event);
   
   event.future().whenComplete((completedFetches, error) -> {
   fetchResults.addAll(completedFetches);
   });
   }
   ```
   
   Since the `whenComplete()` method is executed when the background thread 
"completes" the `Future`, `fetchResults` is thus modified on the background 
thread.
   
   The rest of the process should occur on the application thread.
   
   During calls to `poll()` on the application thread, data from `fetchResults` 
is moved to `fetchBuffer` in `pollForFetches()`:
   
   ```java
   while (pollTimer.notExpired()) {
   CompletedFetch completedFetch = 
fetchResults.poll(pollTimer.remainingMs(), TimeUnit.MILLISECONDS);
   
   if (completedFetch != null)
   fetchBuffer.add(completedFetch);
   
   pollTimer.update();
   }
   ```
   
   The data in `fetchBuffer` is later extracted in `fetchCollector` during the 
`poll()` process, but this again is on the application thread.
   
   This roundabout way of getting the data is specifically done so that we 
don't write to the `FetchBuffer` inadvertently from the background thread. 
Hence these JavaDoc comment for `fetchResults`:
   
   ```java
   /**
* A thread-safe {@link BlockingQueue queue} for the results that are 
populated in the background thread
* when the fetch results are available. Because the {@link #fetchBuffer 
fetch buffer} is not thread-safe, we
* need to separate the results collection that we provide to the background 
thread from the collection that
* we read from on the application thread.
*/
   private final BlockingQueue fetchResults = new 
LinkedBlockingQueue<>();
   ```
   
   This is a rough idea of what happens on the background thread:
   
   >`FetchRequestManager` -> `fetchResults`
   
   Then later in the application thread during `poll()`:
   
   > `fetchResults` -> `fetchBuffer` -> `fetchCollector`
   
   Let me know if that makes sense or if there is still a gap that I'm not 
seeing. I can write the above up (with any changes you'd like) in code comments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344899034


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -395,12 +523,12 @@ private void maybeThrowInvalidGroupIdException() {
 
 @Override
 public Map metrics() {
-throw new KafkaException("method not implemented");
+return Collections.unmodifiableMap(this.metrics.metrics());

Review Comment:
   I've removed the qualifiers where they're not needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-03 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1344900260


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -489,20 +623,44 @@ private Map 
beginningOrEndOffset(Collection timestampToSearch =
-
partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> 
timestamp));
+Map timestampToSearch = partitions
+.stream()
+.collect(Collectors.toMap(Function.identity(), tp -> 
timestamp));
 final 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()));
+false
+);

Review Comment:
   Fixed.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -489,20 +623,44 @@ private Map 
beginningOrEndOffset(Collection timestampToSearch =
-
partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> 
timestamp));
+Map timestampToSearch = partitions
+.stream()
+.collect(Collectors.toMap(Function.identity(), tp -> 
timestamp));
 final 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()));
+false
+);

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


junrao commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346179200


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -37,7 +36,7 @@
  *
  * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
  */
-public class FetchBuffer implements Closeable {
+public class FetchBuffer implements AutoCloseable {

Review Comment:
   @kirktrue : Thanks for the explanation. Two followup questions.
   1. The background thread has the following path` FetchRequestManager.poll  
-> handleFetchResponse -> fetchBuffer.add(completedFetch)`. So, it seems that 
the background thread also writes the fetched data to `fetchBuffer`.
   2. This is related to my other 
[comment](https://github.com/apache/kafka/pull/14406#discussion_r1342984775). 
The background thread has the following path `FetchRequestManager.poll  -> 
AbstractFetch.prepareFetchRequests -> AbstractFetch.fetchablePartitions -> 
reads fetchBuffer.bufferedPartitions()`. Since fetchBuffer is written by the 
application thread, how do we coordinate the synchronization btw the two 
threads?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346234546


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##
@@ -222,58 +99,158 @@ public void run() {
 } 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());
 }
 }
 
+void initializeResources() {
+applicationEventProcessor = applicationEventProcessorSupplier.get();
+networkClientDelegate = networkClientDelegateSupplier.get();
+requestManagers = requestManagersSupplier.get();
+}
+
 /**
- * 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.
+ * Poll and process the {@link ApplicationEvent application events}. It 
performs the following tasks:
+ *
+ * 
+ * 
+ * Drains and processes all the events from the application 
thread's application event queue via
+ * {@link ApplicationEventProcessor}
+ * 
+ * 
+ * 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
+ * 
+ * 
+ * Stage each {@link AbstractRequest.Builder request} to be sent 
via
+ * {@link NetworkClientDelegate#addAll(List)}
+ * 
+ * 
+ * Poll the client via {@link KafkaClient#poll(long, long)} to 
send the requests, as well as
+ * retrieve any available responses
+ * 
+ * 
  */
 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);
-}
-}
+// 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(m -> m.get().poll(currentTimeMs))
-.map(this::handlePollResult)
+.map(Optional::get)
+.map(rm -> rm.poll(currentTimeMs))
+.map(networkClientDelegate::addAll)
 .reduce(MAX_POLL_TIMEOUT_MS, Math::min);
 networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs);
 }
 
-long handlePollResult(NetworkClientDelegate.PollResult res) {
-if (!res.unsentRequests.isEmpty()) {
-networkClientDelegate.addAll(res.unsentRequests);
+/**
+ * Performs any network I/O that is needed at the time of close for the 
consumer:
+ *
+ * 
+ * 
+ * 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
+ * 
+ * 
+ * Stage each {@link AbstractRequest.Builder request} to be sent 
via
+ * {@link NetworkClientDelegate#addAll(List)}
+ * 
+ * 
+ * {@link KafkaClient#poll(long, long) Poll the client} to send 
the requests, as well as
+ * retrieve any available responses
+ * 
+ * 
+ * Continuously {@link KafkaClient#poll(long, long) poll the 
client} as long as the
+ * {@link Timer#notExpired() timer hasn't expired} to retrieve the 
responses
+ * 
+ * 
+ */
+// Visible for testing
+static void runAtClose(final Time time,
+   final Collection> requestManagers,
+   final NetworkClientDelegate networkClientDelegate,
+   final Timer timer) {
+long currentTimeMs = time.milliseconds();
+
+// These are the optional outgoing requests at the
+List pollResults = 
requestManagers.stream()
+.filter(Optional::isPresent)
+.map(Optional::get)
+.map(RequestManager::pollOnClose)
+.collect(Collectors.toList());

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346270372


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##
@@ -222,58 +99,158 @@ public void run() {
 } 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());
 }
 }
 
+void initializeResources() {
+applicationEventProcessor = applicationEventProcessorSupplier.get();
+networkClientDelegate = networkClientDelegateSupplier.get();
+requestManagers = requestManagersSupplier.get();
+}
+
 /**
- * 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.
+ * Poll and process the {@link ApplicationEvent application events}. It 
performs the following tasks:
+ *
+ * 
+ * 
+ * Drains and processes all the events from the application 
thread's application event queue via
+ * {@link ApplicationEventProcessor}
+ * 
+ * 
+ * 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
+ * 
+ * 
+ * Stage each {@link AbstractRequest.Builder request} to be sent 
via
+ * {@link NetworkClientDelegate#addAll(List)}
+ * 
+ * 
+ * Poll the client via {@link KafkaClient#poll(long, long)} to 
send the requests, as well as
+ * retrieve any available responses
+ * 
+ * 
  */
 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);
-}
-}
+// If there are errors processing any events, the error will be thrown 
immediately. This will have
+// the effect of closing the background thread.

Review Comment:
   Yes. I added a check that it's not closed at the top of `runOnce()`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346278813


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java:
##
@@ -0,0 +1,77 @@
+/*
+ * 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.DefaultBackgroundThread;
+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  {@link 
DefaultBackgroundThread the background thread}.
+ * Those events are generally of two types:
+ *
+ * 
+ * Errors that occur in the background 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 
DefaultBackgroundThread background thread}.
+ * It is possible that when processing the events that a given event will
+ * {@link ErrorBackgroundEvent represent an error directly}, or it could 
be that processing an event generates
+ * an error. In such cases, the processor will continue to process the 
remaining events. In this case, we
+ * provide the caller to provide a callback handler that "collects" the 
errors. We grab the first error that

Review Comment:
   This was an outdated comment. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


philipnee commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346493970


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##
@@ -106,7 +127,62 @@ public void onFailure(RuntimeException e) {
 return fetchRequestMap.size();
 }
 
+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);

Review Comment:
   Thanks @kirktrue - I think we will need to do that.  I created KAFKA-15548 
to handle the closing task.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346504058


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java:
##
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+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.NetworkClient;
+import 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
+import 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest;
+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.requests.FetchRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+/**
+ * {@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 Logger log;
+private final BackgroundEventHandler backgroundEventHandler;
+private final NetworkClientDelegate networkClientDelegate;
+private final List>> futures;
+
+FetchRequestManager(final LogContext logContext,
+final Time time,
+final BackgroundEventHandler backgroundEventHandler,
+final ConsumerMetadata metadata,
+final SubscriptionState subscriptions,
+final FetchConfig fetchConfig,
+final FetchMetricsManager metricsManager,
+final NetworkClientDelegate networkClientDelegate) {
+super(logContext, metadata, subscriptions, fetchConfig, 
metricsManager, time);
+this.log = logContext.logger(FetchRequestManager.class);
+this.backgroundEventHandler = backgroundEventHandler;
+this.networkClientDelegate = networkClientDelegate;
+this.futures = new ArrayList<>();
+}
+
+@Override
+protected boolean isUnavailable(Node node) {
+return networkClientDelegate.isUnavailable(node);
+}
+
+@Override
+protected void maybeThrowAuthFailure(Node node) {
+networkClientDelegate.maybeThrowAuthFailure(node);
+}
+
+/**
+ * Adds a new {@link Future future} to the list of futures awaiting 
results. Per the comments on
+ * {@link #forwardResults()}, there is no guarantee that this particular 
future will be provided with
+ * a non-empty result, but it is guaranteed to be completed with a result, 
assuming that it does not time out.
+ *
+ * @param future Future that will be {@link 
CompletableFuture#complete(Object) completed} if not timed out
+ */
+public void requestFetch(CompletableFuture> future) {
+futures.add(future);
+}
+
+@Override
+public PollResult poll(long currentTimeMs) {
+List requests = 
prepareFetchRequests().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, t) -> {
+if (t != null) {
+handleFetchResponse(fetchTarget, t);
+log.debug("Attempt to fetch data from node {} failed due 
to fatal exception", fetchTarget, t);
+   

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346508562


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -693,15 +1015,14 @@ private boolean refreshCommittedOffsetsIfNeeded(Timer 
timer) {
 
 log.debug("Refreshing committed offsets for partitions {}", 
initializingPartitions);
 try {
-final Map offsets = 
eventHandler.addAndGet(new OffsetFetchApplicationEvent(initializingPartitions), 
timer);
+final Map offsets = 
applicationEventHandler.addAndGet(new 
OffsetFetchApplicationEvent(initializingPartitions), timer);

Review Comment:
   @lianetm Any reason we don't check if `initializingPartitions` is non-empty 
before creating the `OffsetFetchApplicationEvent`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346517075


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void se

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346545044


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -37,7 +36,7 @@
  *
  * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
  */
-public class FetchBuffer implements Closeable {
+public class FetchBuffer implements AutoCloseable {

Review Comment:
   For point #1, the background thread has a _separate_ fetch buffer. It 
doesn't write to the same object. The application thread `FetchBuffer` is 
created in `PrototypeAsyncConsumer` and the background thread `FetchBuffer` is 
created in `AbstractFetch`.
   
   For point #2, I think your [other 
comment](https://github.com/apache/kafka/pull/14406#discussion_r1342984775) is 
correct. I'm not yet sure how to maintain those two fetch buffers separately 
without running into that race condition.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346546088


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void se

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346604856


##
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.DefaultBackgroundThread;
+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 DefaultBackgroundThread background 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 
DefaultBackgroundThread background thread}
+ */
+public void add(BackgroundEvent event) {
+Objects.requireNonNull(event, "BackgroundEvent provided to add must be 
non-null");
+log.trace("Enqueued event: {}", event);

Review Comment:
   Yes, it would, but what do you expect when you're logging at `TRACE`? 😛 



##
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.DefaultBackgroundThread;
+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 DefaultBackgroundThread background 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 
DefaultBackgroundThread background thread}
+ */
+public void add(BackgroundEvent event) {
+Objects.requireNonNull(event, "BackgroundEvent provided to add must be 
non-null");
+log.trace("Enqueued event: {}", event);

Review Comment:
   Yes, it would, but what do you expect when you're logging at `TRACE`? 😛 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346606768


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java:
##
@@ -0,0 +1,122 @@
+/*
+ * 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.DefaultBackgroundThread;
+import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate;
+import org.apache.kafka.clients.consumer.internals.RequestManagers;
+import org.apache.kafka.common.KafkaException;
+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.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 
background thread.
+ */
+public class ApplicationEventHandler implements Closeable {
+
+private final Logger log;
+private final BlockingQueue applicationEventQueue;
+private final DefaultBackgroundThread backgroundThread;
+private final IdempotentCloser closer = new IdempotentCloser();
+
+public ApplicationEventHandler(final Time time,
+   final LogContext logContext,
+   final BlockingQueue 
applicationEventQueue,
+   final Supplier 
applicationEventProcessorSupplier,
+   final Supplier 
networkClientDelegateSupplier,
+   final Supplier 
requestManagersSupplier) {
+this.log = logContext.logger(ApplicationEventHandler.class);
+this.applicationEventQueue = applicationEventQueue;
+this.backgroundThread = new DefaultBackgroundThread(time,
+logContext,
+applicationEventProcessorSupplier,
+networkClientDelegateSupplier,
+requestManagersSupplier);
+this.backgroundThread.start();
+}
+
+/**
+ * Add an {@link ApplicationEvent} to the handler.
+ *
+ * @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);
+backgroundThread.wakeup();
+applicationEventQueue.add(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
+ * @paramType 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.ofMillis(Long.MAX_VALUE));
+}
+
+public void close(final Duration timeout) {
+Objects.requireNonNull(timeout, "Duration provided to close must be 
non-null");
+
+closer.close(
+() ->  {
+   

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346606932


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##
@@ -106,7 +127,62 @@ public void onFailure(RuntimeException e) {
 return fetchRequestMap.size();
 }
 
+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);

Review Comment:
   I'm hoping to split this into a separate Jira, if possible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346607807


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java:
##
@@ -0,0 +1,136 @@
+/*
+ * 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.LinkedList;
+import java.util.List;
+import java.util.Queue;
+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;
+import org.slf4j.Logger;
+
+/**
+ * {@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 Logger log;
+private final ErrorEventHandler errorEventHandler;
+private final NetworkClientDelegate networkClientDelegate;
+
+FetchRequestManager(final LogContext logContext,
+final Time time,
+final ErrorEventHandler errorEventHandler,
+final ConsumerMetadata metadata,
+final SubscriptionState subscriptions,
+final FetchConfig fetchConfig,
+final FetchMetricsManager metricsManager,
+final NetworkClientDelegate networkClientDelegate) {
+super(logContext, metadata, subscriptions, fetchConfig, 
metricsManager, time);
+this.log = logContext.logger(FetchRequestManager.class);
+this.errorEventHandler = errorEventHandler;
+this.networkClientDelegate = networkClientDelegate;
+}
+
+@Override
+protected boolean isUnavailable(Node node) {
+return networkClientDelegate.isUnavailable(node);
+}
+
+@Override
+protected void maybeThrowAuthFailure(Node node) {
+networkClientDelegate.maybeThrowAuthFailure(node);
+}
+
+@Override
+public PollResult poll(long currentTimeMs) {
+List requests;
+
+if (!idempotentCloser.isClosed()) {
+// If the fetcher is open (i.e. not closed), we will issue the 
normal fetch requests
+requests = prepareFetchRequests().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, t) -> {
+if (t != null) {
+handleFetchResponse(fetchTarget, t);
+log.warn("Attempt to fetch data from node {} failed 
due to fatal exception", fetchTarget, t);
+errorEventHandler.handle(t);

Review Comment:
   I've removed the logging and error forwarding as it is handled in the 
`NetworkClientDelegate` layer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346612374


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346613925


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -622,12 +837,18 @@ public void assign(Collection partitions) 
{
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance

Review Comment:
   @junrao I'll review with @philipnee.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346615044


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##
@@ -92,6 +88,23 @@ public AbstractFetch(final LogContext logContext,
 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);

Review Comment:
   @philipnee `AbstractFetch` is used by both the current `Fetcher` as well as 
the `FetchRequestManager`. I'd have to refactor the code to rid ourselves of it.
   
   What do you think about filing a new Jira to track a follow-up change for 
this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346616102


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-04 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346616646


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void se

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


lianetm commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347752851


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void sen

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


lianetm commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r134973


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void sen

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347799078


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347874026


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -693,15 +1015,14 @@ private boolean refreshCommittedOffsetsIfNeeded(Timer 
timer) {
 
 log.debug("Refreshing committed offsets for partitions {}", 
initializingPartitions);
 try {
-final Map offsets = 
eventHandler.addAndGet(new OffsetFetchApplicationEvent(initializingPartitions), 
timer);
+final Map offsets = 
applicationEventHandler.addAndGet(new 
OffsetFetchApplicationEvent(initializingPartitions), timer);

Review Comment:
   TODO: find case where we _might_ be sending RPCs with empty sets.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -622,12 +837,18 @@ public void assign(Collection partitions) 
{
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance

Review Comment:
   TODO: fix the comment in the code and explain about the 
ApplicationEventProcessor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347874693


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void se

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


lianetm commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347885711


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -693,15 +1015,14 @@ private boolean refreshCommittedOffsetsIfNeeded(Timer 
timer) {
 
 log.debug("Refreshing committed offsets for partitions {}", 
initializingPartitions);
 try {
-final Map offsets = 
eventHandler.addAndGet(new OffsetFetchApplicationEvent(initializingPartitions), 
timer);
+final Map offsets = 
applicationEventHandler.addAndGet(new 
OffsetFetchApplicationEvent(initializingPartitions), timer);

Review Comment:
   I know for sure we skip empty lists and don't send requests for all the 
partition offsets related events (on the `OffsetsRequestManager` on 
[fetchOffsets](https://github.com/apache/kafka/blob/cdf726fd358f9be3438ceefb01073ab40a31a8b4/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java#L151),
 
[resetPositions](https://github.com/apache/kafka/blob/cdf726fd358f9be3438ceefb01073ab40a31a8b4/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java#L193)
 and 
[validatePositions](https://github.com/apache/kafka/blob/cdf726fd358f9be3438ceefb01073ab40a31a8b4/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java#L216)).
 
   That being said, here is about fetching the committed offsets, and I don't 
see a clear early return but maybe I'm missing how it happens so let's wait for 
@philipnee to give it a closer look. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


lianetm commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1347752851


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void sen

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348009426


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348009622


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348011353


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final ConsumerConfig config,
 @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));
-}
+backgroundEventProcessor.process();
 
-updateFetchPositionsIfNeeded(timer);
+this.kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
 
-// 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);
-}
-// 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();
+if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {
+throw new IllegalStateException("Consumer is not subscribed to 
any topics or assigned any partitions");
+}
 
-return ConsumerRecords.empty();
-}
+do {
+updateAssignmentMetadataIfNeeded(timer);
+final Fetch fetch = pollForFetches(timer);
 
-/**
- * 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) {
-// If any partitions have been truncated due to a leader change, we 
need to validate the offsets
-ValidatePositionsApplicationEvent validatePositionsEvent = new 
ValidatePositionsApplicationEvent();
-eventHandler.add(validatePositionsEvent);
+if (!fetch.isEmpty()) {
+sendFetches();
 
-// If there are any partitions which do not have a valid position and 
are not
-// awaiting reset, then we need to fetch committed offsets. We 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 (fetch.records().isEmpty()) {
+log.trace("Returning empty records from `poll()` "
++ "since the consumer's position has advanced 
for at least one topic partition");
+}
 
-// 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();
+return this.interceptors.onConsume(new 
ConsumerRecords<>(fetch.records()));
+}
+// We will wait for retryBackoffMs
+} while (

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348029408


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##
@@ -92,6 +88,23 @@ public AbstractFetch(final LogContext logContext,
 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);

Review Comment:
   I have filed KAFKA-15556 (_Remove NetworkClientDelegate methods 
isUnavailable, maybeThrowAuthFailure, and tryConnect_) to address this issue 
since it's affecting other `RequestManager` implementations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348029671


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##
@@ -330,7 +343,7 @@ private Map 
prepareCloseFetchSession
 // skip sending the close request.
 final Node fetchTarget = cluster.nodeById(fetchTargetNodeId);
 
-if (fetchTarget == null || client.isUnavailable(fetchTarget)) {
+if (fetchTarget == null || isUnavailable(fetchTarget)) {

Review Comment:
   True. We'll handle that in KAFKA-15556.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1346606932


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##
@@ -106,7 +127,62 @@ public void onFailure(RuntimeException e) {
 return fetchRequestMap.size();
 }
 
+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);

Review Comment:
   I'm hoping to split this into a separate Jira, if possible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348047927


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3565 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348055761


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,174 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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()));
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
 log.info("Assigned to partition(s): {}", join(partitions, ", "));
 if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+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);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the background thread and set up to 
collect the results in
+ * {@link #fetchResults}.
+ */
+private void se

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348062596


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -693,15 +1015,14 @@ private boolean refreshCommittedOffsetsIfNeeded(Timer 
timer) {
 
 log.debug("Refreshing committed offsets for partitions {}", 
initializingPartitions);
 try {
-final Map offsets = 
eventHandler.addAndGet(new OffsetFetchApplicationEvent(initializingPartitions), 
timer);
+final Map offsets = 
applicationEventHandler.addAndGet(new 
OffsetFetchApplicationEvent(initializingPartitions), timer);

Review Comment:
   @philipnee filed KAFKA-15551 (_Evaluate conditions for short circuiting 
consumer API calls_) to implement this consistently.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348083698


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -636,42 +857,148 @@ public void assign(Collection 
partitions) {
 }
 
 @Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-throw new KafkaException("method not implemented");
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().equals(""))
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.metadata.requestUpdateForNewTopics();
+}
+
+/**
+ * TODO: remove this when we implement the KIP-848 protocol.
+ *
+ * 
+ * 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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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 void sendFetches() {
+FetchEvent event = new FetchEvent();
+eventHandler.add(event);
+
+event.future().whenComplete((completedFetches, error) -> {
+if (completedFetches != null && !completedFetches.isEmpty()) {
+fetchBuffer.addAll(completedFetches);
+}
+});
+}
+
+/**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+private Fetch pollForFetches(Timer timer) {
+long pollTimeout = timer.remainingMs();
+
+// if data is available already, return it immediately
+final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
+if (!fetch.isEmpty()) {
+return fetch;
+}
+
+// send any new fetches (won't resend pending fetches)
+sendFetches();
+
+// We do not want to be stuck blocking in poll if we are missing some 
positions
+// since the offset lookup may be backing off after a failure
+
+// NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+// updateAssignmentMetadataIfNeeded before this method.
+if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+pollTimeout = retryBackoffMs;
+}
+
+log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+Timer pollTimer = time.timer(pollTimeout);
+
+// Attempt to fetch any data. It's OK if we time out here; it's a best 
case effort. The
+// data may not be immediately available, but the calling method 
(poll) will correctly
+// handle the overall timeout.
+try {
+Queue completedFetches = 
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+if (completedFetches != null && !completedFetches.isEmpty()) {
+ 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348084921


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -636,42 +857,148 @@ public void assign(Collection 
partitions) {
 }
 
 @Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-throw new KafkaException("method not implemented");
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().equals(""))
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.metadata.requestUpdateForNewTopics();
+}
+
+/**
+ * TODO: remove this when we implement the KIP-848 protocol.
+ *
+ * 
+ * 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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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 void sendFetches() {
+FetchEvent event = new FetchEvent();
+eventHandler.add(event);
+
+event.future().whenComplete((completedFetches, error) -> {
+if (completedFetches != null && !completedFetches.isEmpty()) {
+fetchBuffer.addAll(completedFetches);
+}
+});
+}
+
+/**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+private Fetch pollForFetches(Timer timer) {
+long pollTimeout = timer.remainingMs();
+
+// if data is available already, return it immediately
+final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
+if (!fetch.isEmpty()) {
+return fetch;
+}
+
+// send any new fetches (won't resend pending fetches)
+sendFetches();
+
+// We do not want to be stuck blocking in poll if we are missing some 
positions
+// since the offset lookup may be backing off after a failure
+
+// NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+// updateAssignmentMetadataIfNeeded before this method.
+if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+pollTimeout = retryBackoffMs;
+}
+
+log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+Timer pollTimer = time.timer(pollTimeout);
+
+// Attempt to fetch any data. It's OK if we time out here; it's a best 
case effort. The
+// data may not be immediately available, but the calling method 
(poll) will correctly
+// handle the overall timeout.
+try {
+Queue completedFetches = 
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+if (completedFetches != null && !completedFetches.isEmpty()) {
+ 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1348087535


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -636,42 +857,148 @@ public void assign(Collection 
partitions) {
 }
 
 @Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-throw new KafkaException("method not implemented");
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().equals(""))
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.metadata.requestUpdateForNewTopics();
+}
+
+/**
+ * TODO: remove this when we implement the KIP-848 protocol.
+ *
+ * 
+ * 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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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 void sendFetches() {
+FetchEvent event = new FetchEvent();
+eventHandler.add(event);
+
+event.future().whenComplete((completedFetches, error) -> {
+if (completedFetches != null && !completedFetches.isEmpty()) {
+fetchBuffer.addAll(completedFetches);
+}
+});
+}
+
+/**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+private Fetch pollForFetches(Timer timer) {
+long pollTimeout = timer.remainingMs();
+
+// if data is available already, return it immediately
+final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
+if (!fetch.isEmpty()) {
+return fetch;
+}
+
+// send any new fetches (won't resend pending fetches)
+sendFetches();
+
+// We do not want to be stuck blocking in poll if we are missing some 
positions
+// since the offset lookup may be backing off after a failure
+
+// NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+// updateAssignmentMetadataIfNeeded before this method.
+if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+pollTimeout = retryBackoffMs;
+}
+
+log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+Timer pollTimer = time.timer(pollTimeout);
+
+// Attempt to fetch any data. It's OK if we time out here; it's a best 
case effort. The
+// data may not be immediately available, but the calling method 
(poll) will correctly
+// handle the overall timeout.
+try {
+Queue completedFetches = 
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+if (completedFetches != null && !completedFetches.isEmpty()) {
+ 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-06 Thread via GitHub


philipnee commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1349221937


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java:
##
@@ -0,0 +1,252 @@
+/*
+ * 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.InterruptException;
+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.Optional;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+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();
+
+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...

Review Comment:
   hi jun - I think we mostly use "background thread" in lieu of network 
thread, so maybe just use background thread?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-06 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1349342751


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -636,42 +857,148 @@ public void assign(Collection 
partitions) {
 }
 
 @Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-throw new KafkaException("method not implemented");
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().equals(""))
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.metadata.requestUpdateForNewTopics();
+}
+
+/**
+ * TODO: remove this when we implement the KIP-848 protocol.
+ *
+ * 
+ * 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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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 void sendFetches() {
+FetchEvent event = new FetchEvent();
+eventHandler.add(event);
+
+event.future().whenComplete((completedFetches, error) -> {
+if (completedFetches != null && !completedFetches.isEmpty()) {
+fetchBuffer.addAll(completedFetches);
+}
+});
+}
+
+/**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+private Fetch pollForFetches(Timer timer) {
+long pollTimeout = timer.remainingMs();
+
+// if data is available already, return it immediately
+final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
+if (!fetch.isEmpty()) {
+return fetch;
+}
+
+// send any new fetches (won't resend pending fetches)
+sendFetches();
+
+// We do not want to be stuck blocking in poll if we are missing some 
positions
+// since the offset lookup may be backing off after a failure
+
+// NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+// updateAssignmentMetadataIfNeeded before this method.
+if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+pollTimeout = retryBackoffMs;
+}
+
+log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+Timer pollTimer = time.timer(pollTimeout);
+
+// Attempt to fetch any data. It's OK if we time out here; it's a best 
case effort. The
+// data may not be immediately available, but the calling method 
(poll) will correctly
+// handle the overall timeout.
+try {
+Queue completedFetches = 
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+if (completedFetches != null && !completedFetches.isEmpty()) {
+ 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-06 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1349343753


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java:
##
@@ -0,0 +1,252 @@
+/*
+ * 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.InterruptException;
+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.Optional;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+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();
+
+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...

Review Comment:
   @philipnee You didn't see that I got fed up and changed the name of 
`DefaultBackgroundThread` to `ConsumerNetworkThread` 😆 



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java:
##
@@ -0,0 +1,252 @@
+/*
+ * 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 ap

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-06 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1349343412


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -37,7 +36,7 @@
  *
  * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
  */
-public class FetchBuffer implements Closeable {
+public class FetchBuffer implements AutoCloseable {

Review Comment:
   I just pushed a proposed fix for this, which is basically to make the 
`FetchBuffer` thread safe. Now there is only one fetch buffer for the consumer 
and is accessed by both threads.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


junrao commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1350697908


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,163 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the {@link ConsumerNetworkThread 
network thread} and set up to
+ * collect 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1351011656


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+import java.util.function.BiConsumer;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+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.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;
+private final List>> futures;
+
+FetchRequestManager(final LogContext logContext,
+final Time time,
+final ConsumerMetadata metadata,
+final SubscriptionState subscriptions,
+final FetchConfig fetchConfig,
+final FetchMetricsManager metricsManager,
+final NetworkClientDelegate networkClientDelegate) {
+super(logContext, metadata, subscriptions, fetchConfig, 
metricsManager, time);
+this.networkClientDelegate = networkClientDelegate;
+this.futures = new ArrayList<>();
+}
+
+@Override
+protected boolean isUnavailable(Node node) {
+return networkClientDelegate.isUnavailable(node);
+}
+
+@Override
+protected void maybeThrowAuthFailure(Node node) {
+networkClientDelegate.maybeThrowAuthFailure(node);
+}
+
+/**
+ * Adds a new {@link Future future} to the list of futures awaiting 
results. Per the comments on
+ * {@link #handleFetchResponse(Node, FetchSessionHandler.FetchRequestData, 
ClientResponse)}}, there is no
+ * guarantee that this particular future will be provided with a non-empty 
result, but it is guaranteed
+ * to be completed with a result, assuming that it does not time out.
+ *
+ * @param future Future that will be {@link 
CompletableFuture#complete(Object) completed} if not timed out
+ */
+public void requestFetch(CompletableFuture> future) {
+futures.add(future);
+}
+
+/**
+ * {@inheritDoc}
+ */
+@Override
+public PollResult poll(long currentTimeMs) {
+return pollInternal(
+prepareFetchRequests(),
+this::handleFetchResponse,
+this::handleFetchResponse
+);
+}
+
+/**
+ * {@inheritDoc}
+ */
+@Override
+public PollResult pollOnClose() {
+return pollInternal(
+prepareCloseFetchSessionRequests(),
+this::handleCloseFetchSessionResponse,
+this::handleCloseFetchSessionResponse
+);
+}
+
+/**
+ * Creates the {@link PollResult poll result} that contains a list of zero 
or more
+ * {@link FetchRequest.Builder fetch requests}  fetch request},
+ * {@link NetworkClient#send(ClientRequest, long) enqueues/sends it, and 
adds the {@link RequestFuture callback}

Review Comment:
   I removed the latter phrases to reduce confusion.



##
clients/src/main/java/org/apache/kafka/clients/consumer/intern

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1351042141


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java:
##
@@ -0,0 +1,288 @@
+/*
+ * 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.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.CommitApplicationEvent;
+import org.apache.kafka.clients.consumer.internals.events.FetchEvent;
+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.errors.TopicAuthorizationException;
+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.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.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+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.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 CoordinatorRequestManager coordinatorManager;
+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;
+coordinatorManager = testBuilder.coordinatorRequestManager;
+commitManager = testBuilder.commitRequestManager;
+offsetsRequestManager = testBuilder.offsetsRequestManager;
+consumerNetworkThread = testBuilder.consumerN

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1351054656


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java:
##
@@ -0,0 +1,288 @@
+/*
+ * 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.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.CommitApplicationEvent;
+import org.apache.kafka.clients.consumer.internals.events.FetchEvent;
+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.errors.TopicAuthorizationException;
+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.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.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+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.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 CoordinatorRequestManager coordinatorManager;
+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;
+coordinatorManager = testBuilder.coordinatorRequestManager;
+commitManager = testBuilder.commitRequestManager;
+offsetsRequestManager = testBuilder.offsetsRequestManager;
+consumerNetworkThread = testBuilder.consumerN

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1351063405


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java:
##
@@ -0,0 +1,288 @@
+/*
+ * 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.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.CommitApplicationEvent;
+import org.apache.kafka.clients.consumer.internals.events.FetchEvent;
+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.errors.TopicAuthorizationException;
+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.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.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+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.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 CoordinatorRequestManager coordinatorManager;
+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;
+coordinatorManager = testBuilder.coordinatorRequestManager;
+commitManager = testBuilder.commitRequestManager;
+offsetsRequestManager = testBuilder.offsetsRequestManager;
+consumerNetworkThread = testBuilder.consumerN

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-09 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1351064502


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.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;
+
+public class ConsumerTestBuilder implements Closeable {
+
+static final long RETRY_BACKOFF_MS = 80;
+static final long RETRY_BACKOFF_MAX_MS = 1000;
+static final int REQUEST_TIMEOUT_MS = 500;
+
+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 Metrics metrics;
+final FetchMetricsManager metricsManager;
+final NetworkClientDelegate networkClientDelegate;
+final OffsetsRequestManager offsetsRequestManager;
+final CoordinatorRequestManager coordinatorRequestManager;
+final CommitRequestManager commitRequestManager;
+final TopicMetadataRequestManager topicMetadataRequestManager;
+final FetchRequestManager fetchRequestManager;
+final RequestManagers requestManagers;
+public final ApplicationEventProcessor applicationEventProcessor;
+public final BackgroundEventProcessor backgroundEventProcessor;
+public final BackgroundEventHandler backgroundEventHandler;
+final MockClient client;
+
+public ConsumerTestBuilder() {
+this.applicationEventQueue = new LinkedBlockingQueue<>();
+this.backgroundEventQueue = new LinkedBlockingQueue<>();
+this.backgroundEventHandler = new BackgroundEventHandler(logContext, 
backgroundEventQueue);
+GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
+100,
+100,
+100,
+"group_id",
+   

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-10 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1352874538


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java:
##
@@ -0,0 +1,103 @@
+/*
+ * 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.LogContext;
+import org.slf4j.Logger;
+
+import java.util.LinkedList;
+import java.util.Objects;
+import java.util.concurrent.BlockingQueue;
+
+public class BackgroundEventProcessor {
+
+private final Logger log;
+private final BlockingQueue backgroundEventQueue;
+
+public BackgroundEventProcessor(final LogContext logContext,
+final BlockingQueue 
backgroundEventQueue) {
+this.log = logContext.logger(BackgroundEventProcessor.class);
+this.backgroundEventQueue = backgroundEventQueue;
+}
+
+/**
+ * Drains all available {@link BackgroundEvent}s, and then processes them 
in order. If any
+ * errors are thrown as a result of a {@link ErrorBackgroundEvent} or an 
error occurs while processing
+ * another type of {@link BackgroundEvent}, only the first 
exception will be thrown, all
+ * subsequent errors will simply be logged at WARN level.
+ *
+ * @throws RuntimeException or subclass
+ */
+public void process() {
+LinkedList events = new LinkedList<>();
+backgroundEventQueue.drainTo(events);
+
+RuntimeException first = null;
+int errorCount = 0;
+
+for (BackgroundEvent event : events) {
+log.debug("Consuming background event: {}", event);
+
+try {
+process(event);
+} catch (RuntimeException e) {
+errorCount++;
+
+if (first == null) {
+first = e;
+log.warn("Error #{} from background thread (will be logged 
and thrown): {}", errorCount, e.getMessage(), e);

Review Comment:
   Removed check to avoid.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##
@@ -16,197 +16,78 @@
  */
 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.clients.consumer.internals.events.CompletableApplicationEvent;
 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.internals.IdempotentCloser;
 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.io.Closeable;
+import java.util.ArrayList;
 import java.util.LinkedList;
+import java.util.List;
 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;
+import java.util.function.Supplier;
 
 /**
  * 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 ref

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-10 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1352876631


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java:
##
@@ -0,0 +1,252 @@
+/*
+ * 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.InterruptException;
+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.Optional;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+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();
+
+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);
+}
+}
+
+void initializeResources() {
+applicationEventProcessor = applicationEventProcessorSupplier.get();
+networkClientDelegate = networkClientDelegateSupplier.get();
+requestM

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-10 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1352879009


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManager.java:
##
@@ -16,13 +16,48 @@
  */
 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.
+ * Because the {@code poll} method is called within the single-threaded 
context of the consumer's main network
+ * I/O thread, there should be no need for synchronization protection 
within itself or other state.
+ *
+ * 
+ *
+ * Note: no network I/O occurs in this method. The method itself 
should not block on I/O or for any
+ * other reason. This method is called from by the consumer's main network 
I/O thread. So quick execution of

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358651225


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358658187


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358665833


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358682245


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358693714


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358701620


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358712167


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358713529


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358717315


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358758478


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java:
##
@@ -1107,7 +1107,7 @@ public void testFetchMaxPollRecords() {
 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, 4), 
fullFetchResponse(tidp0, nextRecords, Errors.NONE, 100L, 0));

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358761112


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358764452


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358791361


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358914146


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358914547


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358918492


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358949767


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358953070


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358953131


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358953536


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358954499


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358955393


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358956176


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358956462


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358956804


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358957417


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358990414


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##
@@ -17,146 +17,195 @@
 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 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 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.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 
immediately throw an exception, and not
+ * process the remaining events.
+ */
+@Override
+public void process() {
+process(error -> {
+throw 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 FETCH:
+process((FetchEvent) event);
+return;
+
 case RESET_POSITIONS:
-return processResetPositionsEvent();
+processResetPositionsEvent();
+return;
+
 case VALIDATE_POSITIONS:
-return processValidatePositionsEvent();
+processValidatePositionsEvent();
+return;
+
+default:
+throw new IllegalArgumentException("Application event type " + 
event.type() + " was not expected");
 }
-return false;
 }
 
-/**
- * Processes {@link NoopApplicationEvent} and enqueue a
- * {@link N

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358990537


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,163 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the {@link ConsumerNetworkThread 
network thread} and set up to
+ * collec

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358990617


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -621,56 +825,163 @@ public void assign(Collection 
partitions) {
 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
 }
 
-// TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set 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
 WakeupTrigger wakeupTrigger() {
 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);
+/**
+ * Send the requests for fetch data to the {@link ConsumerNetworkThread 
network thread} and set up to
+ * collec

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-13 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1358990797


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -66,31 +82,107 @@ boolean isEmpty() {
  * @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:
+ *
+ * 
+ * The buffer was already non-empty on entry
+ * The buffer was populated during the wait
+ * The remaining time on the {@link Timer timer} elapsed
+ * The thread was interrupted
+ * 
+ *
+ * @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)) {

Review Comment:
   Thanks for catching that. I fixed this but it's pretty clear now that I need 
unit tests to validate correctness.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-16 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1361329900


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -37,7 +36,7 @@
  *
  * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
  */
-public class FetchBuffer implements Closeable {
+public class FetchBuffer implements AutoCloseable {

Review Comment:
   @junrao this is one of the more serious and subtle issues, so I want to know 
you're OK with the new approach before I resolve this conversation in the PR. 
Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-16 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1361329379


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -636,42 +857,148 @@ public void assign(Collection 
partitions) {
 }
 
 @Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-throw new KafkaException("method not implemented");
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().equals(""))
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+this.subscriptions.subscribe(pattern, listener);
+this.updatePatternSubscription(metadata.fetch());
+this.metadata.requestUpdateForNewTopics();
+}
+
+/**
+ * TODO: remove this when we implement the KIP-848 protocol.
+ *
+ * 
+ * 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());
+this.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
 WakeupTrigger wakeupTrigger() {
 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 void sendFetches() {
+FetchEvent event = new FetchEvent();
+eventHandler.add(event);
+
+event.future().whenComplete((completedFetches, error) -> {
+if (completedFetches != null && !completedFetches.isEmpty()) {
+fetchBuffer.addAll(completedFetches);
+}
+});
+}
+
+/**
+ * @throws KafkaException if the rebalance callback throws exception
+ */
+private Fetch pollForFetches(Timer timer) {
+long pollTimeout = timer.remainingMs();
+
+// if data is available already, return it immediately
+final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
+if (!fetch.isEmpty()) {
+return fetch;
+}
+
+// send any new fetches (won't resend pending fetches)
+sendFetches();
+
+// We do not want to be stuck blocking in poll if we are missing some 
positions
+// since the offset lookup may be backing off after a failure
+
+// NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+// updateAssignmentMetadataIfNeeded before this method.
+if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+pollTimeout = retryBackoffMs;
+}
+
+log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+Timer pollTimer = time.timer(pollTimeout);
+
+// Attempt to fetch any data. It's OK if we time out here; it's a best 
case effort. The
+// data may not be immediately available, but the calling method 
(poll) will correctly
+// handle the overall timeout.
+try {
+Queue completedFetches = 
eventHandler.addAndGet(new FetchEvent(), pollTimer);
+if (completedFetches != null && !completedFetches.isEmpty()) {
+ 

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-16 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1361332784


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-16 Thread via GitHub


kirktrue commented on code in PR #14406:
URL: https://github.com/apache/kafka/pull/14406#discussion_r1361345756


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##
@@ -0,0 +1,3573 @@
+/*
+ * 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.com

  1   2   >