apoorvmittal10 commented on code in PR #20852:
URL: https://github.com/apache/kafka/pull/20852#discussion_r2524246289


##########
core/src/test/java/kafka/server/NetworkPartitionMetadataClientTest.java:
##########
@@ -0,0 +1,706 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.MockClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.MockTime;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class NetworkPartitionMetadataClientTest {
+    private static final MockTime MOCK_TIME = new MockTime();
+    private static final MetadataCache METADATA_CACHE = 
mock(MetadataCache.class);
+    private static final KafkaClient CLIENT = mock(KafkaClient.class);
+    private static final String HOST = "localhost";
+    private static final int PORT = 9092;
+    private static final ListenerName LISTENER_NAME = 
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT);
+    private static final String TOPIC = "test-topic";
+    private static final int PARTITION = 0;
+    private static final Node LEADER_NODE = new Node(1, HOST, PORT);
+    private static final long REQUEST_TIMEOUT_MS = 
CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS;
+    
+    private NetworkPartitionMetadataClient networkPartitionMetadataClient;
+
+    private static class NetworkPartitionMetadataClientBuilder {
+        private MetadataCache metadataCache = METADATA_CACHE;
+        private KafkaClient client = CLIENT;
+        private Time time = MOCK_TIME;
+        private ListenerName listenerName = LISTENER_NAME;
+
+        NetworkPartitionMetadataClientBuilder withMetadataCache(MetadataCache 
metadataCache) {
+            this.metadataCache = metadataCache;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withKafkaClient(KafkaClient 
client) {
+            this.client = client;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withListenerName(ListenerName 
listenerName) {
+            this.listenerName = listenerName;
+            return this;
+        }
+
+        public static NetworkPartitionMetadataClientBuilder bulider() {
+            return new NetworkPartitionMetadataClientBuilder();
+        }
+
+        public NetworkPartitionMetadataClient build() {

Review Comment:
   ```suggestion
           NetworkPartitionMetadataClient build() {
   ```



##########
core/src/test/java/kafka/server/NetworkPartitionMetadataClientTest.java:
##########
@@ -0,0 +1,706 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.MockClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.MockTime;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class NetworkPartitionMetadataClientTest {
+    private static final MockTime MOCK_TIME = new MockTime();
+    private static final MetadataCache METADATA_CACHE = 
mock(MetadataCache.class);
+    private static final KafkaClient CLIENT = mock(KafkaClient.class);
+    private static final String HOST = "localhost";
+    private static final int PORT = 9092;
+    private static final ListenerName LISTENER_NAME = 
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT);
+    private static final String TOPIC = "test-topic";
+    private static final int PARTITION = 0;
+    private static final Node LEADER_NODE = new Node(1, HOST, PORT);
+    private static final long REQUEST_TIMEOUT_MS = 
CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS;
+    
+    private NetworkPartitionMetadataClient networkPartitionMetadataClient;
+
+    private static class NetworkPartitionMetadataClientBuilder {
+        private MetadataCache metadataCache = METADATA_CACHE;
+        private KafkaClient client = CLIENT;
+        private Time time = MOCK_TIME;
+        private ListenerName listenerName = LISTENER_NAME;
+
+        NetworkPartitionMetadataClientBuilder withMetadataCache(MetadataCache 
metadataCache) {
+            this.metadataCache = metadataCache;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withKafkaClient(KafkaClient 
client) {
+            this.client = client;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withTime(Time time) {
+            this.time = time;
+            return this;

Review Comment:
   Are these methods used ever? If not then remove them.



##########
core/src/test/java/kafka/server/NetworkPartitionMetadataClientTest.java:
##########
@@ -0,0 +1,706 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.MockClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.MockTime;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class NetworkPartitionMetadataClientTest {
+    private static final MockTime MOCK_TIME = new MockTime();
+    private static final MetadataCache METADATA_CACHE = 
mock(MetadataCache.class);
+    private static final KafkaClient CLIENT = mock(KafkaClient.class);
+    private static final String HOST = "localhost";
+    private static final int PORT = 9092;
+    private static final ListenerName LISTENER_NAME = 
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT);
+    private static final String TOPIC = "test-topic";
+    private static final int PARTITION = 0;
+    private static final Node LEADER_NODE = new Node(1, HOST, PORT);
+    private static final long REQUEST_TIMEOUT_MS = 
CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS;
+    
+    private NetworkPartitionMetadataClient networkPartitionMetadataClient;
+
+    private static class NetworkPartitionMetadataClientBuilder {
+        private MetadataCache metadataCache = METADATA_CACHE;
+        private KafkaClient client = CLIENT;
+        private Time time = MOCK_TIME;
+        private ListenerName listenerName = LISTENER_NAME;
+
+        NetworkPartitionMetadataClientBuilder withMetadataCache(MetadataCache 
metadataCache) {
+            this.metadataCache = metadataCache;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withKafkaClient(KafkaClient 
client) {
+            this.client = client;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        NetworkPartitionMetadataClientBuilder withListenerName(ListenerName 
listenerName) {
+            this.listenerName = listenerName;
+            return this;
+        }
+
+        public static NetworkPartitionMetadataClientBuilder bulider() {

Review Comment:
   ```suggestion
           static NetworkPartitionMetadataClientBuilder builder() {
   ```



##########
core/src/test/java/kafka/server/NetworkPartitionMetadataClientTest.java:
##########
@@ -0,0 +1,706 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.MockClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.MockTime;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class NetworkPartitionMetadataClientTest {

Review Comment:
   Missing tests
   - Failure handling of condition: `clientResponse.wasTimedOut()` and 
`clientResponse == null`
   - Close method error condition - `catch (InterruptedException e)`



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########


Review Comment:
   Do we need catch block now?



##########
core/src/main/java/kafka/server/NetworkPartitionMetadataClient.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+
+public class NetworkPartitionMetadataClient implements PartitionMetadataClient 
{
+
+    private static final Logger log = 
LoggerFactory.getLogger(NetworkPartitionMetadataClient.class);
+
+    private final MetadataCache metadataCache;
+    private final Supplier<KafkaClient> networkClientSupplier;
+    private volatile SendThread sendThread;
+    private final Time time;
+    private final ListenerName listenerName;
+    private final Object initializationLock = new Object();
+
+    public NetworkPartitionMetadataClient(
+        MetadataCache metadataCache,
+        Supplier<KafkaClient> networkClientSupplier,
+        Time time,
+        ListenerName listenerName
+    ) {
+        this.metadataCache = metadataCache;
+        this.networkClientSupplier = networkClientSupplier;
+        this.time = time;
+        this.listenerName = listenerName;
+        this.sendThread = null;
+    }
+
+    @Override
+    public Map<TopicPartition, CompletableFuture<OffsetResponse>> 
listLatestOffsets(
+        Set<TopicPartition> topicPartitions
+    ) {
+        if (topicPartitions == null || topicPartitions.isEmpty()) {
+            return Map.of();
+        }
+
+        // Initialize sendThread lazily on first call
+        ensureSendThreadInitialized();
+
+        // Map to store futures for each TopicPartition
+        Map<TopicPartition, CompletableFuture<OffsetResponse>> futures = new 
HashMap<>();
+        // Group TopicPartitions by leader node
+        Map<Node, List<TopicPartition>> partitionsByNode = new HashMap<>();
+        for (TopicPartition tp : topicPartitions) {
+            // Get leader node for this partition
+            Optional<Node> leaderNodeOpt = 
metadataCache.getPartitionLeaderEndpoint(
+                tp.topic(),
+                tp.partition(),
+                listenerName
+            );
+
+            if (leaderNodeOpt.isEmpty() || leaderNodeOpt.get().isEmpty()) {
+                // No leader available - complete with error
+                futures.put(tp, CompletableFuture.completedFuture(new 
OffsetResponse(-1, Errors.LEADER_NOT_AVAILABLE)));
+                continue;
+            }
+
+            partitionsByNode.computeIfAbsent(leaderNodeOpt.get(), k -> new 
ArrayList<>()).add(tp);
+        }
+
+        // Create and enqueue requests for each node
+        partitionsByNode.forEach((node, partitionsByLeader) -> {
+            // All partitions with the same leader node will be included in 
the same ListOffsetsRequest.
+            Map<TopicPartition, CompletableFuture<OffsetResponse>> 
partitionFuturesByLeader = new HashMap<>();
+            for (TopicPartition tp : partitionsByLeader) {
+                CompletableFuture<OffsetResponse> future = new 
CompletableFuture<>();
+                futures.put(tp, future);
+                partitionFuturesByLeader.put(tp, future);
+            }
+
+            // Create ListOffsetsRequest for this node
+            ListOffsetsRequest.Builder requestBuilder = 
createListOffsetsRequest(partitionsByLeader);
+            // Create pending request to track this request
+            PendingRequest pendingRequest = new PendingRequest(node, 
partitionFuturesByLeader, requestBuilder);
+            // Enqueue to send thread
+            sendThread.enqueue(pendingRequest);
+        });
+
+        return futures;
+    }
+
+
+    @Override
+    public void close() {
+        // Only close sendThread if it was initialized. Note, clos is called 
only during broker shutdown, so need
+        // for further synchronization here.
+        if (sendThread != null) {
+            try {
+                sendThread.shutdown();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                log.error("Interrupted while shutting down 
NetworkPartitionMetadataClient", e);
+            }
+        }
+    }
+
+    /**
+     * Ensures that the sendThread is initialized. This method is thread-safe 
and will only
+     * initialize the sendThread once, even if called concurrently.
+     */
+    // Visible for testing.
+    void ensureSendThreadInitialized() {
+        if (sendThread == null) {
+            synchronized (initializationLock) {
+                if (sendThread == null) {
+                    KafkaClient networkClient = networkClientSupplier.get();
+                    SendThread thread = new SendThread(
+                        "NetworkPartitionMetadataClientSendThread",
+                        networkClient,
+                        
Math.toIntExact(CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS),
  //30 seconds
+                        this.time
+                    );
+                    thread.start();
+                    sendThread = thread;
+                    log.debug("NetworkPartitionMetadataClient sendThread 
initialized and started");

Review Comment:
   ```suggestion
                       log.info("NetworkPartitionMetadataClient sendThread 
initialized and started");
   ```



##########
core/src/main/java/kafka/server/NetworkPartitionMetadataClient.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+
+public class NetworkPartitionMetadataClient implements PartitionMetadataClient 
{
+
+    private static final Logger log = 
LoggerFactory.getLogger(NetworkPartitionMetadataClient.class);
+
+    private final MetadataCache metadataCache;
+    private final Supplier<KafkaClient> networkClientSupplier;
+    private volatile SendThread sendThread;

Review Comment:
   nit: move after the final variable declaration



##########
core/src/main/java/kafka/server/NetworkPartitionMetadataClient.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+
+public class NetworkPartitionMetadataClient implements PartitionMetadataClient 
{
+
+    private static final Logger log = 
LoggerFactory.getLogger(NetworkPartitionMetadataClient.class);
+
+    private final MetadataCache metadataCache;
+    private final Supplier<KafkaClient> networkClientSupplier;
+    private volatile SendThread sendThread;
+    private final Time time;
+    private final ListenerName listenerName;
+    private final Object initializationLock = new Object();
+
+    public NetworkPartitionMetadataClient(
+        MetadataCache metadataCache,
+        Supplier<KafkaClient> networkClientSupplier,
+        Time time,
+        ListenerName listenerName
+    ) {
+        this.metadataCache = metadataCache;
+        this.networkClientSupplier = networkClientSupplier;
+        this.time = time;
+        this.listenerName = listenerName;
+        this.sendThread = null;

Review Comment:
   nit: not required



##########
core/src/main/java/kafka/server/NetworkPartitionMetadataClient.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.network.ListenerName;
+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.utils.Time;
+import org.apache.kafka.coordinator.group.PartitionMetadataClient;
+import org.apache.kafka.metadata.MetadataCache;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+
+public class NetworkPartitionMetadataClient implements PartitionMetadataClient 
{
+
+    private static final Logger log = 
LoggerFactory.getLogger(NetworkPartitionMetadataClient.class);
+
+    private final MetadataCache metadataCache;
+    private final Supplier<KafkaClient> networkClientSupplier;
+    private volatile SendThread sendThread;
+    private final Time time;
+    private final ListenerName listenerName;
+    private final Object initializationLock = new Object();
+
+    public NetworkPartitionMetadataClient(
+        MetadataCache metadataCache,
+        Supplier<KafkaClient> networkClientSupplier,
+        Time time,
+        ListenerName listenerName
+    ) {
+        this.metadataCache = metadataCache;
+        this.networkClientSupplier = networkClientSupplier;
+        this.time = time;
+        this.listenerName = listenerName;
+        this.sendThread = null;
+    }
+
+    @Override
+    public Map<TopicPartition, CompletableFuture<OffsetResponse>> 
listLatestOffsets(
+        Set<TopicPartition> topicPartitions
+    ) {
+        if (topicPartitions == null || topicPartitions.isEmpty()) {
+            return Map.of();
+        }
+
+        // Initialize sendThread lazily on first call
+        ensureSendThreadInitialized();
+
+        // Map to store futures for each TopicPartition
+        Map<TopicPartition, CompletableFuture<OffsetResponse>> futures = new 
HashMap<>();
+        // Group TopicPartitions by leader node
+        Map<Node, List<TopicPartition>> partitionsByNode = new HashMap<>();
+        for (TopicPartition tp : topicPartitions) {
+            // Get leader node for this partition
+            Optional<Node> leaderNodeOpt = 
metadataCache.getPartitionLeaderEndpoint(
+                tp.topic(),
+                tp.partition(),
+                listenerName
+            );
+
+            if (leaderNodeOpt.isEmpty() || leaderNodeOpt.get().isEmpty()) {
+                // No leader available - complete with error
+                futures.put(tp, CompletableFuture.completedFuture(new 
OffsetResponse(-1, Errors.LEADER_NOT_AVAILABLE)));
+                continue;
+            }
+
+            partitionsByNode.computeIfAbsent(leaderNodeOpt.get(), k -> new 
ArrayList<>()).add(tp);
+        }
+
+        // Create and enqueue requests for each node
+        partitionsByNode.forEach((node, partitionsByLeader) -> {
+            // All partitions with the same leader node will be included in 
the same ListOffsetsRequest.
+            Map<TopicPartition, CompletableFuture<OffsetResponse>> 
partitionFuturesByLeader = new HashMap<>();
+            for (TopicPartition tp : partitionsByLeader) {
+                CompletableFuture<OffsetResponse> future = new 
CompletableFuture<>();
+                futures.put(tp, future);
+                partitionFuturesByLeader.put(tp, future);
+            }
+
+            // Create ListOffsetsRequest for this node
+            ListOffsetsRequest.Builder requestBuilder = 
createListOffsetsRequest(partitionsByLeader);
+            // Create pending request to track this request
+            PendingRequest pendingRequest = new PendingRequest(node, 
partitionFuturesByLeader, requestBuilder);
+            // Enqueue to send thread
+            sendThread.enqueue(pendingRequest);
+        });
+
+        return futures;
+    }
+
+
+    @Override
+    public void close() {
+        // Only close sendThread if it was initialized. Note, clos is called 
only during broker shutdown, so need
+        // for further synchronization here.
+        if (sendThread != null) {
+            try {
+                sendThread.shutdown();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                log.error("Interrupted while shutting down 
NetworkPartitionMetadataClient", e);
+            }
+        }
+    }
+
+    /**
+     * Ensures that the sendThread is initialized. This method is thread-safe 
and will only
+     * initialize the sendThread once, even if called concurrently.
+     */
+    // Visible for testing.
+    void ensureSendThreadInitialized() {
+        if (sendThread == null) {
+            synchronized (initializationLock) {
+                if (sendThread == null) {

Review Comment:
   Better to use AtomicBoolean with `compareAndSet`.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to