Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MINOR: Enhance performance of ConsumerRecords by refactoring iterator initialization and iteration logic #16494

Open
wants to merge 16 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
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
Expand Down Expand Up @@ -56,7 +57,7 @@ public List<ConsumerRecord<K, V>> records(TopicPartition partition) {
/**
* Get just the records for the given topic
*/
public Iterable<ConsumerRecord<K, V>> records(String topic) {
public Iterable<ConsumerRecord<K, V>> recordsWithNestedList(String topic) {
if (topic == null)
throw new IllegalArgumentException("Topic must be non-null.");
List<List<ConsumerRecord<K, V>>> recs = new ArrayList<>();
Expand All @@ -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));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we don't need to use ConcatenatedIterable. for example:

    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();
            }
        };
    }

}

/**
* Get the partitions which have records contained in this record set.
* @return the set of partitions with data in this record set (may be empty if no data was returned)
Expand All @@ -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) {
chia7712 marked this conversation as resolved.
Show resolved Hide resolved
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

this.iterables = iterables;
this.predicate = predicate;
}

@Override
public Iterator<ConsumerRecord<K, V>> iterator() {
return new AbstractIterator<ConsumerRecord<K, V>>() {
final Iterator<? extends Iterable<ConsumerRecord<K, V>>> iters = iterables.iterator();
Iterator<ConsumerRecord<K, V>> current;

protected ConsumerRecord<K, V> makeNext() {
while (current == null || !current.hasNext()) {
if (iters.hasNext())
current = iters.next().iterator();
else
return allDone();
while (true) {
if (current == null || !current.hasNext()) {
if (!advanceToNextIterator()) {
return allDone();
}
}

ConsumerRecord<K, V> next = current.next();

if (predicate == null || predicate.test(next)) {
return next;
}
}
}

private boolean advanceToNextIterator() {
while (iters.hasNext()) {
current = iters.next().iterator();
if (current.hasNext()) {
return true;
}
}
return current.next();
return false;
}
};
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not moving this new method to origin ConsumerRecords?

if (topic == null)
throw new IllegalArgumentException("Topic must be non-null.");
return new ConcatenatedIterable<>(records, record -> record.topic().equals(topic));
}

/**
* Get the partitions which have records contained in this record set.
*
* @return the set of partitions with data in this record set (may be empty if no data was returned)
*/
public Set<TopicPartition> partitions() {
return Collections.unmodifiableSet(records.keySet());
}

@Override
public Iterator<ConsumerRecord<K, V>> iterator() {
return new ConcatenatedIterable<>(records).iterator();
}

/**
* The number of records for all topics
*/
public int count() {
int count = 0;
for (List<ConsumerRecord<K, V>> recs : this.records.values())
count += recs.size();
return count;
}

private static class ConcatenatedIterable<K, V> implements Iterable<ConsumerRecord<K, V>> {

private final Map<TopicPartition, ? extends Iterable<ConsumerRecord<K, V>>> topicPartitionToRecords;
private final Predicate<TopicPartition> predicate;

public ConcatenatedIterable(Map<TopicPartition, ? extends Iterable<ConsumerRecord<K, V>>> topicPartitionToRecords) {
this(topicPartitionToRecords, null);
}

public ConcatenatedIterable(Map<TopicPartition, ? extends Iterable<ConsumerRecord<K, V>>> topicPartitionToRecords, Predicate<TopicPartition> predicate) {
this.topicPartitionToRecords = topicPartitionToRecords;
this.predicate = predicate;
}

@Override
public Iterator<ConsumerRecord<K, V>> iterator() {
return new AbstractIterator<ConsumerRecord<K, V>>() {
final Iterator<? extends Map.Entry<TopicPartition, ? extends Iterable<ConsumerRecord<K, V>>>> iterator
= topicPartitionToRecords.entrySet().iterator();
Iterator<ConsumerRecord<K, V>> current;

protected ConsumerRecord<K, V> makeNext() {
while (current == null || !current.hasNext()) {
if (!advanceToNextIterator()) {
return allDone();
}
}
return current.next();
}

private boolean advanceToNextIterator() {
while (iterator.hasNext()) {
Map.Entry<TopicPartition, ? extends Iterable<ConsumerRecord<K, V>>> next = iterator.next();
if (predicate == null || predicate.test(next.getKey())) {
current = next.getValue().iterator();
return true;
}
}
return false;
}
};
}
}

public boolean isEmpty() {
return records.isEmpty();
}

@SuppressWarnings("unchecked")
public static <K, V> ConsumerRecordsNew<K, V> empty() {
return (ConsumerRecordsNew<K, V>) EMPTY;
}

}
Loading