Copilot commented on code in PR #17633:
URL: https://github.com/apache/pinot/pull/17633#discussion_r2768602893


##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-4.0/src/test/java/org/apache/pinot/plugin/stream/kafka40/utils/MiniKafkaCluster.java:
##########
@@ -0,0 +1,118 @@
+/**
+ * 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.pinot.plugin.stream.kafka40.utils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.TopicPartition;
+import org.testcontainers.kafka.KafkaContainer;
+import org.testcontainers.utility.DockerImageName;
+
+
+/**
+ * MiniKafkaCluster for Kafka 4.x using Testcontainers with KRaft mode (no 
ZooKeeper).
+ * Uses the apache/kafka-native image for fast startup.
+ */
+public final class MiniKafkaCluster implements Closeable {
+
+  private static final String KAFKA_IMAGE = "apache/kafka:4.0.0";
+
+  private final KafkaContainer _kafkaContainer;
+  private final AdminClient _adminClient;

Review Comment:
   The field `_adminClient` is declared but initialized to null in the 
constructor and never assigned a value. It is also checked for null in the 
`close()` method but can never be non-null. This field should either be removed 
or properly initialized if it's needed for future functionality.



##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-4.0/src/main/java/org/apache/pinot/plugin/stream/kafka40/KafkaStreamMetadataProvider.java:
##########
@@ -0,0 +1,287 @@
+/**
+ * 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.pinot.plugin.stream.kafka40;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.time.Clock;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.pinot.plugin.stream.kafka.KafkaConsumerPartitionLag;
+import org.apache.pinot.spi.stream.ConsumerPartitionState;
+import org.apache.pinot.spi.stream.LongMsgOffset;
+import org.apache.pinot.spi.stream.OffsetCriteria;
+import org.apache.pinot.spi.stream.PartitionLagState;
+import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.stream.StreamMessageMetadata;
+import org.apache.pinot.spi.stream.StreamMetadataProvider;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.stream.TransientConsumerException;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class KafkaStreamMetadataProvider extends 
KafkaPartitionLevelConnectionHandler
+    implements StreamMetadataProvider {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+
+  public KafkaStreamMetadataProvider(String clientId, StreamConfig 
streamConfig) {
+    this(clientId, streamConfig, Integer.MIN_VALUE);
+  }
+
+  public KafkaStreamMetadataProvider(String clientId, StreamConfig 
streamConfig, int partition) {
+    super(clientId, streamConfig, partition);
+  }
+
+  @Override
+  public int fetchPartitionCount(long timeoutMillis) {
+    try {
+      List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic, 
Duration.ofMillis(timeoutMillis));
+      if (CollectionUtils.isNotEmpty(partitionInfos)) {
+        return partitionInfos.size();
+      }
+      throw new RuntimeException(String.format("Failed to fetch partition 
information for topic: %s", _topic));
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Set<Integer> fetchPartitionIds(long timeoutMillis) {
+    try {
+      List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic, 
Duration.ofMillis(timeoutMillis));
+      if (CollectionUtils.isEmpty(partitionInfos)) {
+        throw new RuntimeException(String.format("Failed to fetch partition 
information for topic: %s", _topic));
+      }
+      Set<Integer> partitionIds = 
Sets.newHashSetWithExpectedSize(partitionInfos.size());
+      for (PartitionInfo partitionInfo : partitionInfos) {
+        partitionIds.add(partitionInfo.partition());
+      }
+      return partitionIds;
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Map<Integer, StreamPartitionMsgOffset> 
fetchLatestStreamOffset(Set<Integer> partitionIds, long timeoutMillis) {
+    List<TopicPartition> topicPartitions = new 
ArrayList<>(partitionIds.size());
+    for (Integer streamPartition: partitionIds) {
+      topicPartitions.add(new TopicPartition(_topic, streamPartition));
+    }
+    try {
+      Map<TopicPartition, Long> topicPartitionToLatestOffsetMap =
+          _consumer.endOffsets(topicPartitions, 
Duration.ofMillis(timeoutMillis));
+
+      Map<Integer, StreamPartitionMsgOffset> partitionIdToLatestOffset =
+          new HashMap<>(topicPartitionToLatestOffsetMap.size());
+      for (Map.Entry<TopicPartition, Long> entry : 
topicPartitionToLatestOffsetMap.entrySet()) {
+        partitionIdToLatestOffset.put(entry.getKey().partition(), new 
LongMsgOffset(entry.getValue()));
+      }
+
+      return partitionIdToLatestOffset;
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public StreamPartitionMsgOffset fetchStreamPartitionOffset(OffsetCriteria 
offsetCriteria, long timeoutMillis) {
+    Preconditions.checkNotNull(offsetCriteria);
+    long offset;
+    try {
+      if (offsetCriteria.isLargest()) {
+        offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+            .get(_topicPartition);
+      } else if (offsetCriteria.isSmallest()) {
+        offset =
+            
_consumer.beginningOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+                .get(_topicPartition);
+      } else if (offsetCriteria.isPeriod()) {
+        OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Collections.singletonMap(_topicPartition,
+                Clock.systemUTC().millis() - 
TimeUtils.convertPeriodToMillis(offsetCriteria.getOffsetString())))
+            .get(_topicPartition);
+        if (offsetAndTimestamp == null) {
+          offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+              .get(_topicPartition);
+          LOGGER.warn(
+              "initial offset type is period and its value evaluates to null 
hence proceeding with offset {} for "
+                  + "topic {} partition {}", offset, _topicPartition.topic(), 
_topicPartition.partition());
+        } else {
+          offset = offsetAndTimestamp.offset();
+        }
+      } else if (offsetCriteria.isTimestamp()) {
+        OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Collections.singletonMap(_topicPartition,
+            
TimeUtils.convertTimestampToMillis(offsetCriteria.getOffsetString()))).get(_topicPartition);
+        if (offsetAndTimestamp == null) {
+          offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+              .get(_topicPartition);
+          LOGGER.warn(
+              "initial offset type is timestamp and its value evaluates to 
null hence proceeding with offset {} for "
+                  + "topic {} partition {}", offset, _topicPartition.topic(), 
_topicPartition.partition());
+        } else {
+          offset = offsetAndTimestamp.offset();
+        }
+      } else {
+        throw new IllegalArgumentException("Unknown initial offset value " + 
offsetCriteria);
+      }
+      return new LongMsgOffset(offset);
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PartitionLagState> getCurrentPartitionLagState(
+      Map<String, ConsumerPartitionState> currentPartitionStateMap) {
+    Map<String, PartitionLagState> perPartitionLag = new HashMap<>();
+    for (Map.Entry<String, ConsumerPartitionState> entry : 
currentPartitionStateMap.entrySet()) {
+      ConsumerPartitionState partitionState = entry.getValue();
+      // Compute records-lag
+      StreamPartitionMsgOffset currentOffset = 
partitionState.getCurrentOffset();
+      StreamPartitionMsgOffset upstreamLatest = 
partitionState.getUpstreamLatestOffset();
+      String offsetLagString = "UNKNOWN";
+
+      if (currentOffset instanceof LongMsgOffset && upstreamLatest instanceof 
LongMsgOffset) {
+        long offsetLag = ((LongMsgOffset) upstreamLatest).getOffset() - 
((LongMsgOffset) currentOffset).getOffset();
+        offsetLagString = String.valueOf(offsetLag);
+      }
+
+      // Compute record-availability
+      String availabilityLagMs = "UNKNOWN";
+      StreamMessageMetadata lastProcessedMessageMetadata = 
partitionState.getLastProcessedRowMetadata();
+      if (lastProcessedMessageMetadata != null && 
partitionState.getLastProcessedTimeMs() > 0) {
+        long availabilityLag =
+            partitionState.getLastProcessedTimeMs() - 
lastProcessedMessageMetadata.getRecordIngestionTimeMs();
+        availabilityLagMs = String.valueOf(availabilityLag);
+      }
+
+      perPartitionLag.put(entry.getKey(), new 
KafkaConsumerPartitionLag(offsetLagString, availabilityLagMs));
+    }
+    return perPartitionLag;
+  }
+
+  @Override
+  public List<TopicMetadata> getTopics() {
+    try {
+      AdminClient adminClient = getOrCreateSharedAdminClient();
+      ListTopicsResult result = adminClient.listTopics();
+      if (result == null) {
+        return Collections.emptyList();
+      }
+      return result.names()
+          .get()
+          .stream()
+          .map(topic -> new KafkaTopicMetadata().setName(topic))
+          .collect(Collectors.toList());
+    } catch (ExecutionException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean supportsOffsetLag() {
+    return true;
+  }
+
+  public static class KafkaTopicMetadata implements TopicMetadata {
+    private String _name;
+
+    public String getName() {
+      return _name;
+    }
+
+    public KafkaTopicMetadata setName(String name) {
+      _name = name;
+      return this;
+    }
+  }
+
+
+
+  @Override
+  public StreamPartitionMsgOffset getOffsetAtTimestamp(int partitionId, long 
timestampMillis, long timeoutMillis) {
+    try {
+      OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Map.of(_topicPartition, timestampMillis),
+          Duration.ofMillis(timeoutMillis)).get(_topicPartition);
+      if (offsetAndTimestamp == null) {
+        return null;
+      }
+      return new LongMsgOffset(offsetAndTimestamp.offset());
+    } catch (Exception e) {
+      LOGGER.warn("Failed to get offset at timestamp {} for partition {}", 
timestampMillis, partitionId, e);
+      return null;
+    }
+  }
+
+  @Override
+  public Map<String, StreamPartitionMsgOffset> getStreamStartOffsets() {
+    List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic);
+    Map<TopicPartition, Long> startOffsets = _consumer.beginningOffsets(
+        partitionInfos.stream()
+            .filter(info -> info != null)
+            .map(info -> new TopicPartition(_topic, info.partition()))

Review Comment:
   Filtering for null PartitionInfo objects is unnecessary. The 
`partitionsFor()` method returns a list of non-null PartitionInfo objects. If 
there are no partitions, it returns null or an empty list, but individual 
elements in the list are never null. This null check adds unnecessary overhead 
and can be removed.



##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-4.0/src/main/java/org/apache/pinot/plugin/stream/kafka40/KafkaStreamMetadataProvider.java:
##########
@@ -0,0 +1,287 @@
+/**
+ * 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.pinot.plugin.stream.kafka40;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.time.Clock;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.pinot.plugin.stream.kafka.KafkaConsumerPartitionLag;
+import org.apache.pinot.spi.stream.ConsumerPartitionState;
+import org.apache.pinot.spi.stream.LongMsgOffset;
+import org.apache.pinot.spi.stream.OffsetCriteria;
+import org.apache.pinot.spi.stream.PartitionLagState;
+import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.stream.StreamMessageMetadata;
+import org.apache.pinot.spi.stream.StreamMetadataProvider;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.stream.TransientConsumerException;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class KafkaStreamMetadataProvider extends 
KafkaPartitionLevelConnectionHandler
+    implements StreamMetadataProvider {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+
+  public KafkaStreamMetadataProvider(String clientId, StreamConfig 
streamConfig) {
+    this(clientId, streamConfig, Integer.MIN_VALUE);
+  }
+
+  public KafkaStreamMetadataProvider(String clientId, StreamConfig 
streamConfig, int partition) {
+    super(clientId, streamConfig, partition);
+  }
+
+  @Override
+  public int fetchPartitionCount(long timeoutMillis) {
+    try {
+      List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic, 
Duration.ofMillis(timeoutMillis));
+      if (CollectionUtils.isNotEmpty(partitionInfos)) {
+        return partitionInfos.size();
+      }
+      throw new RuntimeException(String.format("Failed to fetch partition 
information for topic: %s", _topic));
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Set<Integer> fetchPartitionIds(long timeoutMillis) {
+    try {
+      List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic, 
Duration.ofMillis(timeoutMillis));
+      if (CollectionUtils.isEmpty(partitionInfos)) {
+        throw new RuntimeException(String.format("Failed to fetch partition 
information for topic: %s", _topic));
+      }
+      Set<Integer> partitionIds = 
Sets.newHashSetWithExpectedSize(partitionInfos.size());
+      for (PartitionInfo partitionInfo : partitionInfos) {
+        partitionIds.add(partitionInfo.partition());
+      }
+      return partitionIds;
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Map<Integer, StreamPartitionMsgOffset> 
fetchLatestStreamOffset(Set<Integer> partitionIds, long timeoutMillis) {
+    List<TopicPartition> topicPartitions = new 
ArrayList<>(partitionIds.size());
+    for (Integer streamPartition: partitionIds) {
+      topicPartitions.add(new TopicPartition(_topic, streamPartition));
+    }
+    try {
+      Map<TopicPartition, Long> topicPartitionToLatestOffsetMap =
+          _consumer.endOffsets(topicPartitions, 
Duration.ofMillis(timeoutMillis));
+
+      Map<Integer, StreamPartitionMsgOffset> partitionIdToLatestOffset =
+          new HashMap<>(topicPartitionToLatestOffsetMap.size());
+      for (Map.Entry<TopicPartition, Long> entry : 
topicPartitionToLatestOffsetMap.entrySet()) {
+        partitionIdToLatestOffset.put(entry.getKey().partition(), new 
LongMsgOffset(entry.getValue()));
+      }
+
+      return partitionIdToLatestOffset;
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public StreamPartitionMsgOffset fetchStreamPartitionOffset(OffsetCriteria 
offsetCriteria, long timeoutMillis) {
+    Preconditions.checkNotNull(offsetCriteria);
+    long offset;
+    try {
+      if (offsetCriteria.isLargest()) {
+        offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+            .get(_topicPartition);
+      } else if (offsetCriteria.isSmallest()) {
+        offset =
+            
_consumer.beginningOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+                .get(_topicPartition);
+      } else if (offsetCriteria.isPeriod()) {
+        OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Collections.singletonMap(_topicPartition,
+                Clock.systemUTC().millis() - 
TimeUtils.convertPeriodToMillis(offsetCriteria.getOffsetString())))
+            .get(_topicPartition);
+        if (offsetAndTimestamp == null) {
+          offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+              .get(_topicPartition);
+          LOGGER.warn(
+              "initial offset type is period and its value evaluates to null 
hence proceeding with offset {} for "
+                  + "topic {} partition {}", offset, _topicPartition.topic(), 
_topicPartition.partition());
+        } else {
+          offset = offsetAndTimestamp.offset();
+        }
+      } else if (offsetCriteria.isTimestamp()) {
+        OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Collections.singletonMap(_topicPartition,
+            
TimeUtils.convertTimestampToMillis(offsetCriteria.getOffsetString()))).get(_topicPartition);
+        if (offsetAndTimestamp == null) {
+          offset = 
_consumer.endOffsets(Collections.singletonList(_topicPartition), 
Duration.ofMillis(timeoutMillis))
+              .get(_topicPartition);
+          LOGGER.warn(
+              "initial offset type is timestamp and its value evaluates to 
null hence proceeding with offset {} for "
+                  + "topic {} partition {}", offset, _topicPartition.topic(), 
_topicPartition.partition());
+        } else {
+          offset = offsetAndTimestamp.offset();
+        }
+      } else {
+        throw new IllegalArgumentException("Unknown initial offset value " + 
offsetCriteria);
+      }
+      return new LongMsgOffset(offset);
+    } catch (TimeoutException e) {
+      throw new TransientConsumerException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PartitionLagState> getCurrentPartitionLagState(
+      Map<String, ConsumerPartitionState> currentPartitionStateMap) {
+    Map<String, PartitionLagState> perPartitionLag = new HashMap<>();
+    for (Map.Entry<String, ConsumerPartitionState> entry : 
currentPartitionStateMap.entrySet()) {
+      ConsumerPartitionState partitionState = entry.getValue();
+      // Compute records-lag
+      StreamPartitionMsgOffset currentOffset = 
partitionState.getCurrentOffset();
+      StreamPartitionMsgOffset upstreamLatest = 
partitionState.getUpstreamLatestOffset();
+      String offsetLagString = "UNKNOWN";
+
+      if (currentOffset instanceof LongMsgOffset && upstreamLatest instanceof 
LongMsgOffset) {
+        long offsetLag = ((LongMsgOffset) upstreamLatest).getOffset() - 
((LongMsgOffset) currentOffset).getOffset();
+        offsetLagString = String.valueOf(offsetLag);
+      }
+
+      // Compute record-availability
+      String availabilityLagMs = "UNKNOWN";
+      StreamMessageMetadata lastProcessedMessageMetadata = 
partitionState.getLastProcessedRowMetadata();
+      if (lastProcessedMessageMetadata != null && 
partitionState.getLastProcessedTimeMs() > 0) {
+        long availabilityLag =
+            partitionState.getLastProcessedTimeMs() - 
lastProcessedMessageMetadata.getRecordIngestionTimeMs();
+        availabilityLagMs = String.valueOf(availabilityLag);
+      }
+
+      perPartitionLag.put(entry.getKey(), new 
KafkaConsumerPartitionLag(offsetLagString, availabilityLagMs));
+    }
+    return perPartitionLag;
+  }
+
+  @Override
+  public List<TopicMetadata> getTopics() {
+    try {
+      AdminClient adminClient = getOrCreateSharedAdminClient();
+      ListTopicsResult result = adminClient.listTopics();
+      if (result == null) {
+        return Collections.emptyList();
+      }
+      return result.names()
+          .get()
+          .stream()
+          .map(topic -> new KafkaTopicMetadata().setName(topic))
+          .collect(Collectors.toList());
+    } catch (ExecutionException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean supportsOffsetLag() {
+    return true;
+  }
+
+  public static class KafkaTopicMetadata implements TopicMetadata {
+    private String _name;
+
+    public String getName() {
+      return _name;
+    }
+
+    public KafkaTopicMetadata setName(String name) {
+      _name = name;
+      return this;
+    }
+  }
+
+
+
+  @Override
+  public StreamPartitionMsgOffset getOffsetAtTimestamp(int partitionId, long 
timestampMillis, long timeoutMillis) {
+    try {
+      OffsetAndTimestamp offsetAndTimestamp = 
_consumer.offsetsForTimes(Map.of(_topicPartition, timestampMillis),
+          Duration.ofMillis(timeoutMillis)).get(_topicPartition);
+      if (offsetAndTimestamp == null) {
+        return null;
+      }
+      return new LongMsgOffset(offsetAndTimestamp.offset());
+    } catch (Exception e) {
+      LOGGER.warn("Failed to get offset at timestamp {} for partition {}", 
timestampMillis, partitionId, e);
+      return null;
+    }
+  }
+
+  @Override
+  public Map<String, StreamPartitionMsgOffset> getStreamStartOffsets() {
+    List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic);
+    Map<TopicPartition, Long> startOffsets = _consumer.beginningOffsets(
+        partitionInfos.stream()
+            .filter(info -> info != null)
+            .map(info -> new TopicPartition(_topic, info.partition()))
+            .collect(Collectors.toList()));
+    return startOffsets.entrySet().stream().collect(
+        Collectors.toMap(
+            entry -> String.valueOf(entry.getKey().partition()),
+            entry -> new LongMsgOffset(entry.getValue()),
+            (existingValue, newValue) -> newValue
+        ));
+  }
+
+  @Override
+  public Map<String, StreamPartitionMsgOffset> getStreamEndOffsets() {
+    List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic);
+    Map<TopicPartition, Long> startOffsets = _consumer.endOffsets(
+        partitionInfos.stream()
+            .filter(info -> info != null)
+            .map(info -> new TopicPartition(_topic, info.partition()))

Review Comment:
   Same as the previous comment - filtering for null PartitionInfo objects is 
unnecessary. The `partitionsFor()` method returns a list of non-null 
PartitionInfo objects.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to