[GitHub] [kafka] junrao commented on a diff in pull request #14359: KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager

2023-09-11 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -0,0 +1,143 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.internals.IdempotentCloser;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Predicate;
+
+/**
+ * {@code FetchBuffer} buffers up the results from the broker responses as 
they are received. It is essentially a
+ * wrapper around a {@link java.util.Queue} of {@link CompletedFetch}.
+ *
+ * 
+ *
+ * Note: this class is not thread-safe and is intended to only be 
used from a single thread.
+ */
+public class FetchBuffer implements Closeable {
+
+private final Logger log;
+private final ConcurrentLinkedQueue completedFetches;
+private final IdempotentCloser idempotentCloser = new IdempotentCloser();
+
+private CompletedFetch nextInLineFetch;
+
+public FetchBuffer(final LogContext logContext) {
+this.log = logContext.logger(FetchBuffer.class);
+this.completedFetches = new ConcurrentLinkedQueue<>();
+}
+
+/**
+ * Returns {@code true} if there are no completed fetches pending to 
return to the user.
+ *
+ * @return {@code true} if the buffer is empty, {@code false} otherwise
+ */
+boolean isEmpty() {
+return completedFetches.isEmpty();
+}
+
+/**
+ * Return whether we have any completed fetches pending return to the 
user. This method is thread-safe. Has
+ * visibility for testing.
+ *
+ * @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);
+}
+
+void add(CompletedFetch completedFetch) {
+completedFetches.add(completedFetch);
+}
+
+void addAll(Collection completedFetches) {
+this.completedFetches.addAll(completedFetches);
+}
+
+CompletedFetch nextInLineFetch() {
+return nextInLineFetch;
+}
+
+void setNextInLineFetch(CompletedFetch completedFetch) {
+this.nextInLineFetch = completedFetch;
+}
+
+CompletedFetch peek() {
+return completedFetches.peek();
+}
+
+CompletedFetch poll() {
+return completedFetches.poll();
+}
+
+/**
+ * Updates the buffer to retain only the fetch data that corresponds to 
the given partitions. Any previously
+ * {@link CompletedFetch fetched data} is removed if its partition is not 
in the given set of partitions.
+ *
+ * @param partitions {@link Set} of {@link TopicPartition}s for which any 
buffered data should be kept
+ */
+void retainAll(final Set partitions) {
+completedFetches.removeIf(cf -> maybeDrain(partitions, cf));
+
+if (maybeDrain(partitions, nextInLineFetch))
+nextInLineFetch = null;
+}
+
+boolean maybeDrain(final Set partitions, final 
CompletedFetch completedFetch) {
+if (completedFetch != null && 
!partitions.contains(completedFetch.partition)) {
+log.debug("Removing {} from buffered fetch data as it is not in 
the set of partitions to retain ({})", completedFetch.partition, partitions);
+completedFetch.drain();
+return true;
+} else {
+return false;
+}
+}
+
+Set partitions() {

Review Comment:
   partitions => fetchablePartitions?



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

[GitHub] [kafka] junrao commented on a diff in pull request #14359: KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager

2023-09-13 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##
@@ -63,6 +68,31 @@ public NetworkClientDelegate(
 this.unsentRequests = new ArrayDeque<>();
 this.requestTimeoutMs = 
config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
 this.retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+this.tryConnectNodes = new HashSet<>();
+}
+
+@Override
+public boolean isUnavailable(Node node) {
+return NetworkClientUtils.isUnavailable(client, node, time);
+}
+
+@Override
+public void maybeThrowAuthFailure(Node node) {
+NetworkClientUtils.maybeThrowAuthFailure(client, node);
+}
+
+@Override
+public void tryConnect(Node node) {

Review Comment:
   `tryConnect` and `maybeTryConnect` seem highly coupled. Why is only the 
former part of the `NodeStatusDetector` interface? Also, it seems that we have 
used `maybeTryConnect`, but not `tryConnect` in this class.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchUtils.java:
##
@@ -0,0 +1,53 @@
+/*
+ * 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.common.TopicPartition;
+
+/**
+ * {@code FetchUtils} provides a place for disparate parts of the fetch logic 
to live.
+ */
+public class FetchUtils {
+
+/**
+ * Performs two combined actions based on the state related to the {@link 
TopicPartition}:
+ *
+ * 
+ * 
+ * Invokes {@link ConsumerMetadata#requestUpdate(boolean)} to 
signal that the metadata is incorrect and
+ * needs to be updated
+ * 
+ * 
+ * Invokes {@link 
SubscriptionState#clearPreferredReadReplica(TopicPartition)} to clear out any 
read replica
+ * information that may be present.
+ * 
+ * 
+ *
+ * This utility method should be invoked if the client detects (or is told 
by a node in the broker) that an
+ * attempt was made to fetch from a node that isn't the leader or 
preferred replica.
+ *
+ * @param metadata {@link ConsumerMetadata} for which to request an update
+ * @param subscriptions {@link SubscriptionState} to clear any internal 
read replica node
+ * @param topicPartition {@link TopicPartition} for which this state 
change is related
+ */
+static void requestMetadataUpdate(final ConsumerMetadata metadata,
+  final SubscriptionState subscriptions,
+  final TopicPartition topicPartition) {
+metadata.requestUpdate(true);

Review Comment:
   This was set to `false`. Why is it changed to `true`? I think `false` is 
correct since we don't want to reset the metadata retry count when receiving 
retriable error due to stale metadata.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NodeStatusDetector.java:
##
@@ -0,0 +1,58 @@
+/*
+ * 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.NetworkClientUtils;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.errors.AuthenticationExcep

[GitHub] [kafka] junrao commented on a diff in pull request #14359: KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager

2023-09-14 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java:
##
@@ -115,13 +122,17 @@ public void testCommittedTransactionRecordsIncluded() {
 Records rawRecords = 
newTranscactionalRecords(ControlRecordType.COMMIT, numRecords);
 FetchResponseData.PartitionData partitionData = new 
FetchResponseData.PartitionData()
 .setRecords(rawRecords);
-CompletedFetch completedFetch = 
newCompletedFetch(IsolationLevel.READ_COMMITTED,
-OffsetResetStrategy.NONE,
-true,
-0,
+CompletedFetch completedFetch = newCompletedFetch(0,

Review Comment:
   Could we merge the next line in? Ditto for similar places below.



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchBufferTest.java:
##
@@ -0,0 +1,193 @@
+/*
+ * 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.consumer.ConsumerConfig;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+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.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * This tests the {@link FetchBuffer} functionality in addition to what {@link 
FetcherTest} tests during the course
+ * of its tests. One of the main concerns of these tests are that we correctly 
handle both places that data is held
+ * internally:
+ *
+ * 
+ * A special "next in line" buffer
+ * The remainder of the buffers in a queue
+ * 
+ */
+public class FetchBufferTest {
+
+private final Time time = new MockTime(0, 0, 0);
+private final TopicPartition topicAPartition0 = new 
TopicPartition("topic-a", 0);
+private final TopicPartition topicAPartition1 = new 
TopicPartition("topic-a", 1);
+private final TopicPartition topicAPartition2 = new 
TopicPartition("topic-a", 2);
+private final Set allPartitions = 
partitions(topicAPartition0, topicAPartition1, topicAPartition2);
+private LogContext logContext;
+
+private SubscriptionState subscriptions;
+
+private FetchMetricsManager metricsManager;
+
+@BeforeEach
+public void setup() {
+logContext = new LogContext();
+
+Properties p = new Properties();
+p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
+p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+ConsumerConfig config = new ConsumerConfig(p);
+
+subscriptions = createSubscriptionState(config, logContext);
+
+Metrics metrics = createMetrics(config, time);
+metricsManager = createFetchMetricsManager(metrics);
+}
+
+/**
+ * Verifies the basics: we can add buffered data to the queue, peek to 
view them, and poll to remove them.
+ */
+@Test
+publ

[GitHub] [kafka] junrao commented on a diff in pull request #14359: KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager

2023-09-14 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/common/internals/IdempotentCloser.java:
##
@@ -0,0 +1,127 @@
+/*
+ * 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.common.internals;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
+
+/**
+ * {@code IdempotentCloser} encapsulates some basic logic to ensure that a 
given resource is only closed once.
+ * It provides methods to invoke callbacks (via optional {@link Runnable}s) 
for either the initial close
+ * and/or any subsequent closes.
+ *
+ * 
+ *
+ * Here's an example:
+ *
+ * 
+ *
+ * public class MyDataFile implements Closeable {
+ *
+ * private final IdempotentCloser closer = new IdempotentCloser();
+ *
+ * private final File file;
+ *
+ * . . .
+ *
+ * public boolean write() {
+ * closer.maybeThrowIllegalStateException(() -> String.format("Data 
file %s already closed!", file));
+ * writeToFile();
+ * }
+ *
+ *
+ * public boolean isClosed() {
+ * return closer.isClosed();
+ * }
+ *
+ * @Override
+ * public void close() {
+ * Runnable onInitialClose = () -> {
+ * cleanUpFile(file);
+ * log.debug("Data file {} closed", file);
+ * };
+ * Runnable onSubsequentClose = () -> {
+ * log.warn("Data file {} already closed!", file);
+ * };
+ * closer.close(onInitialClose, onSubsequentClose);
+ * }
+ * }
+ * 
+ *
+ * Note that the callbacks are optional and if unused operates as a simple 
means to ensure resources
+ * are only closed once.
+ */
+public class IdempotentCloser implements AutoCloseable {
+
+private final AtomicBoolean isClosed;
+
+/**
+ * Creates an {@code IdempotentCloser} that is not yet closed.
+ */
+public IdempotentCloser() {
+this(false);
+}
+
+/**
+ * Creates an {@code IdempotentCloser} with the given initial state.
+ *
+ * @param isClosed Initial value for underlying state
+ */
+public IdempotentCloser(boolean isClosed) {
+this.isClosed = new AtomicBoolean(isClosed);
+}
+
+public void maybeThrowIllegalStateException(Supplier message) {

Review Comment:
   The comment makes sense. Would `assertOpen` be a clearer method name?



-- 
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