[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-17 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1110454390


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java:
##
@@ -252,6 +230,11 @@ public void teardown() throws Exception {
 }
 }
 
+private int sendFetches() {
+offsetFetcher.validatePositionsOnMetadataChange();

Review Comment:
   Got it. The tests are probably the main reason we called the method from 
`sendFetches`. What we're really depending on is the transition to `Fetching` 
in `SubscriptionState`.



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



[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-17 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1110451696


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java:
##
@@ -0,0 +1,717 @@
+/*
+ * 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.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.clients.StaleMetadataException;
+import org.apache.kafka.clients.consumer.LogTruncationException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import 
org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
+import 
org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+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;
+
+/**
+ * {@link OffsetFetcher} is responsible for fetching the {@link 
OffsetAndTimestamp offsets} for
+ * a given set of {@link TopicPartition topic and partition pairs} and for 
validation and resetting of positions,
+ * as needed.
+ */
+public class OffsetFetcher {
+
+private final Logger log;
+private final ConsumerMetadata metadata;
+private final SubscriptionState subscriptions;
+private final ConsumerNetworkClient client;
+private final Time time;
+private final long retryBackoffMs;
+private final long requestTimeoutMs;
+private final IsolationLevel isolationLevel;
+private final AtomicReference cachedListOffsetsException 
= new AtomicReference<>();
+private final AtomicReference 
cachedOffsetForLeaderException = new AtomicReference<>();
+private final OffsetsForLeaderEpochClient offsetsForLeaderEpochClient;
+private final ApiVersions apiVersions;
+private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1);
+
+public OffsetFetcher(LogContext logContext,
+ ConsumerNetworkClient client,
+ ConsumerMetadata metadata,
+ SubscriptionState subscriptions,
+ Time time,
+ long retryBackoffMs,
+ long requestTimeoutMs,
+ IsolationLevel isolationLevel,
+ ApiVersions apiVersions) {
+this.log = logContext.logger(getClass());
+this.time = time;
+this.client = client;
+this.metadata = metadata;
+this.subscriptions = subscriptions;
+this.retryBackoffMs = 

[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-17 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1110451025


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -1269,6 +1283,11 @@ private ConsumerRecords poll(final Timer timer, 
final boolean includeMetad
 }
 }
 
+private int sendFetches() {
+offsetFetcher.validatePositionsOnMetadataChange();

Review Comment:
   The relocation of this makes me wonder if it's needed at all. We already 
call the same method in `updateFetchPositions`, which is invoked prior to 
`sendFetches`. I tried removing it locally and all tests still pass. Probably 
not a good idea to remove here in the refactor, but maybe we could do it in a 
follow-up. That would simplify the `OffsetFetcher` api a little.



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



[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-17 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1110445520


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -1249,7 +1263,7 @@ private ConsumerRecords poll(final Timer timer, 
final boolean includeMetad
 //
 // NOTE: since the consumed position has already been 
updated, we must not allow

Review Comment:
   I think you could argue the comment is still correct if possibly misleading. 
We track the consumed position in the same field as the fetch position, but we 
have indeed updated it at this point. And we might end up bringing back the old 
field anyway.



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



[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-14 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1106467125


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MetadataFetcher.java:
##
@@ -0,0 +1,805 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.clients.StaleMetadataException;
+import org.apache.kafka.clients.consumer.LogTruncationException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import 
org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
+import 
org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+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;
+
+public class MetadataFetcher {

Review Comment:
   Some high-level documentation for this class would be useful. "Metadata" is 
such an overloaded term that we could probably justify sticking anything in 
here. Would it make sense to call it `OffsetManager` or something like that 
instead? It seems like most of the logic here is offset bookkeeping.



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



[GitHub] [kafka] hachikuji commented on a diff in pull request #13192: KAFKA-14675: Extract metadata-related tasks from Fetcher into MetadataFetcher 1/4

2023-02-14 Thread via GitHub


hachikuji commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1106464011


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MetadataFetcher.java:
##
@@ -0,0 +1,805 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.clients.StaleMetadataException;
+import org.apache.kafka.clients.consumer.LogTruncationException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import 
org.apache.kafka.clients.consumer.internals.OffsetsForLeaderEpochClient.OffsetForEpochResult;
+import 
org.apache.kafka.clients.consumer.internals.SubscriptionState.FetchPosition;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+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;
+
+public class MetadataFetcher {
+
+private final Logger log;
+private final ConsumerMetadata metadata;
+private final SubscriptionState subscriptions;
+private final ConsumerNetworkClient client;
+private final Time time;
+private final long retryBackoffMs;
+private final long requestTimeoutMs;
+private final IsolationLevel isolationLevel;
+private final AtomicReference cachedListOffsetsException 
= new AtomicReference<>();
+private final AtomicReference 
cachedOffsetForLeaderException = new AtomicReference<>();
+private final OffsetsForLeaderEpochClient offsetsForLeaderEpochClient;
+private final ApiVersions apiVersions;
+private final AtomicInteger metadataUpdateVersion = new AtomicInteger(-1);

Review Comment:
   This field is only used in `validatePositionsOnMetadataChange`, which seems 
to be duplicated in both `Fetcher` and `MetadataFetcher`. I think we can drop 
the duplicate method in `Fetcher`. Instead, we can add a method like this to 
`KafkaConsumer`:
   
   ```java
   private int sendFetches() {
 metadataFetcher.validatePositionsOnMetadataChange();
 return fetcher.sendFetches();
   }
   ```
   Does that make sense?
   
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to