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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecordsNew.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.AbstractIterator;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Predicate;
+
+/**
+ * A container that holds the list {@link ConsumerRecord} per partition for a
+ * particular topic. There is one {@link ConsumerRecord} list for every topic
+ * partition returned by a {@link Consumer#poll(java.time.Duration)} operation.
+ */
+public class ConsumerRecordsNew<K, V> implements Iterable<ConsumerRecord<K, 
V>> {
+    public static final ConsumerRecordsNew<Object, Object> EMPTY = new 
ConsumerRecordsNew<>(Collections.emptyMap());
+
+    private final Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
+
+    public ConsumerRecordsNew(Map<TopicPartition, List<ConsumerRecord<K, V>>> 
records) {
+        this.records = records;
+    }
+
+    /**
+     * Get just the records for the given partition
+     *
+     * @param partition The partition to get records for
+     */
+    public List<ConsumerRecord<K, V>> records(TopicPartition partition) {
+        List<ConsumerRecord<K, V>> recs = this.records.get(partition);
+        if (recs == null)
+            return Collections.emptyList();
+        else
+            return Collections.unmodifiableList(recs);
+    }
+
+    public Iterable<ConsumerRecord<K, V>> records(String topic) {

Review Comment:
   why not moving this new method to origin `ConsumerRecords`?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java:
##########
@@ -93,25 +100,47 @@ public int count() {
     private static class ConcatenatedIterable<K, V> implements 
Iterable<ConsumerRecord<K, V>> {
 
         private final Iterable<? extends Iterable<ConsumerRecord<K, V>>> 
iterables;
+        private Predicate<ConsumerRecord<K, V>> predicate = null;
 
         public ConcatenatedIterable(Iterable<? extends 
Iterable<ConsumerRecord<K, V>>> iterables) {
             this.iterables = iterables;
         }
 
+        public ConcatenatedIterable(Iterable<? extends 
Iterable<ConsumerRecord<K, V>>> iterables, Predicate<ConsumerRecord<K, V>> 
predicate) {

Review Comment:
   please remove this version as it has big performance issue, right?



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