-
Notifications
You must be signed in to change notification settings - Fork 13.9k
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
frankvicky
wants to merge
16
commits into
apache:trunk
Choose a base branch
from
frankvicky:consumerrecords-benchmark
base: trunk
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+170
−9
Open
Changes from 6 commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
dbd3e2e
Beanchmark: Stream, forEach
frankvicky 5247a1c
Beanchmark: Custom filterIterator and benchmark
frankvicky 4ab4fd3
Beanchmark: Refactor to avoid double-while loop
frankvicky 2597de3
Beanchmark: Remove unnecessary method.
frankvicky 9eb4e0f
Beanchmark: Use Blackhole to avoid JVM optimizing
frankvicky b64a62e
Beanchmark: Filter Map in the ConcatenatedIterable
frankvicky d607e33
Refacotr new implementation
frankvicky d84af71
Iterate all records in benchmark
frankvicky 2ef1e6e
Add Init benchmark and rename
frankvicky 81e3439
Add new subclass for legacy implementation
frankvicky b974f28
Move legacy implementation to jmh module.
frankvicky 4f834db
Refactor new implementation and improve iterate performance
frankvicky 7ce7a38
Do early return if curretnIterator hasNext is true
frankvicky 1c141ba
Increase warmup iteration
frankvicky 45352f6
Merge branch 'trunk' into consumerrecords-benchmark
frankvicky 8a75217
Merge branch 'trunk' into consumerrecords-benchmark
frankvicky File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
|
@@ -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<>(); | ||
|
@@ -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)); | ||
} | ||
|
||
/** | ||
* 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) | ||
|
@@ -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
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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; | ||
} | ||
}; | ||
} | ||
|
139 changes: 139 additions & 0 deletions
139
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecordsNew.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why not moving this new method to origin |
||
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; | ||
} | ||
|
||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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: