chia7712 commented on code in PR #16494:
URL: https://github.com/apache/kafka/pull/16494#discussion_r1666484737


##########
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java:
##########
@@ -67,6 +68,12 @@ public Iterable<ConsumerRecord<K, V>> records(String topic) {
         return new ConcatenatedIterable<>(recs);
     }
 
+    public Iterable<ConsumerRecord<K, V>> records(String topic) {
+        if (topic == null)
+            throw new IllegalArgumentException("Topic must be non-null.");
+        return new ConcatenatedIterable<>(records.values(), record -> 
record.topic().equals(topic));

Review Comment:
   Maybe we don't need to use `ConcatenatedIterable`. for example:
   ```java
       public Iterable<ConsumerRecord<K, V>> records(String topic) {
           if (topic == null)
               throw new IllegalArgumentException("Topic must be non-null.");
   
           return () -> new AbstractIterator<ConsumerRecord<K, V>>() {
               final Iterator<Map.Entry<TopicPartition, List<ConsumerRecord<K, 
V>>>> iter = records.entrySet().iterator();
               Iterator<ConsumerRecord<K, V>> current = null;
               @Override
               protected ConsumerRecord<K, V> makeNext() {
                   if (current == null || !current.hasNext()) {
                       while (iter.hasNext()) {
                           Map.Entry<TopicPartition, List<ConsumerRecord<K, 
V>>> entry = iter.next();
                           if (entry.getKey().topic().equals(topic) && 
!entry.getValue().isEmpty()) {
                               current = entry.getValue().iterator();
                               break;
                           }
                       }
                   }
                   if (current == null || !current.hasNext()) return allDone();
                   return current.next();
               }
           };
       }
   ```



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

Reply via email to