Skip to content
Closed
Changes from all 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 @@ -472,46 +472,39 @@ public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {

Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<>();
int recordsRemaining = maxPollRecords;
TopicPartition fetchedPartition = null;
long fetchedOffsets = -1L;

try {
while (recordsRemaining > 0) {
if (nextInLineRecords == null || nextInLineRecords.isDrained()) {
CompletedFetch completedFetch = completedFetches.poll();
if (completedFetch == null) break;

fetchedPartition = completedFetch.partition;
fetchedOffsets = completedFetch.fetchedOffset;
while (recordsRemaining > 0) {
if (nextInLineRecords == null || nextInLineRecords.isDrained()) {
CompletedFetch completedFetch = completedFetches.poll();
if (completedFetch == null) break;
try {
nextInLineRecords = parseCompletedFetch(completedFetch);
} else {
TopicPartition partition = nextInLineRecords.partition;
fetchedPartition = partition;
fetchedOffsets = subscriptions.position(partition);
List<ConsumerRecord<K, V>> records = drainRecords(nextInLineRecords, recordsRemaining);
if (!records.isEmpty()) {
List<ConsumerRecord<K, V>> currentRecords = drained.get(partition);
if (currentRecords == null) {
drained.put(partition, records);
} else {
// this case shouldn't usually happen because we only send one fetch at a time per partition,
// but it might conceivably happen in some rare cases (such as partition leader changes).
// we have to copy to a new list because the old one may be immutable
List<ConsumerRecord<K, V>> newRecords = new ArrayList<>(records.size() + currentRecords.size());
newRecords.addAll(currentRecords);
newRecords.addAll(records);
drained.put(partition, newRecords);
}
recordsRemaining -= records.size();
} catch (KafkaException e) {
if (drained.isEmpty())
throw e;
nextInLineExceptionMetadata = new ExceptionMetadata(completedFetch.partition, completedFetch.fetchedOffset, e);
}
} else {
TopicPartition partition = nextInLineRecords.partition;
List<ConsumerRecord<K, V>> records = drainRecords(nextInLineRecords, recordsRemaining);
if (!records.isEmpty()) {
List<ConsumerRecord<K, V>> currentRecords = drained.get(partition);
if (currentRecords == null) {
drained.put(partition, records);
} else {
// this case shouldn't usually happen because we only send one fetch at a time per partition,
// but it might conceivably happen in some rare cases (such as partition leader changes).
// we have to copy to a new list because the old one may be immutable
List<ConsumerRecord<K, V>> newRecords = new ArrayList<>(records.size() + currentRecords.size());
newRecords.addAll(currentRecords);
newRecords.addAll(records);
drained.put(partition, newRecords);
}
recordsRemaining -= records.size();
}
}
} catch (KafkaException e) {
if (drained.isEmpty())
throw e;
nextInLineExceptionMetadata = new ExceptionMetadata(fetchedPartition, fetchedOffsets, e);
}


return drained;
}

Expand Down