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

KafkaConsumer should continue to poll while waiting for buffer #4023

Merged
merged 4 commits into from
Feb 16, 2024
Merged
Changes from 2 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 @@ -75,6 +75,7 @@ public class KafkaCustomConsumer implements Runnable, ConsumerRebalanceListener
private final String topicName;
private final TopicConsumerConfig topicConfig;
private MessageFormat schema;
private boolean paused;
private final BufferAccumulator<Record<Event>> bufferAccumulator;
private final Buffer<Record<Event>> buffer;
private static final ObjectMapper objectMapper = new ObjectMapper();
Expand Down Expand Up @@ -110,6 +111,7 @@ public KafkaCustomConsumer(final KafkaConsumer consumer,
this.shutdownInProgress = shutdownInProgress;
this.consumer = consumer;
this.buffer = buffer;
this.paused = false;
this.byteDecoder = byteDecoder;
this.topicMetrics = topicMetrics;
this.pauseConsumePredicate = pauseConsumePredicate;
Expand Down Expand Up @@ -424,16 +426,25 @@ private void processRecord(final AcknowledgementSet acknowledgementSet, final Re
bufferAccumulator.add(record);
Copy link
Member

Choose a reason for hiding this comment

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

what happens to this record that was caught in exception ? Is this lost ?

Copy link
Member

Choose a reason for hiding this comment

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

I see, the consumer retries in while loop.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

No. The record is put in buffer eventually because we are in an infinite loop here.

break;
} catch (Exception e) {
if (!paused) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I would suggest to pause only when we are approaching max.poll.ms limit, which we can also increase to 10 mins or more. I have seen this exception many times in scale testing, but buffer usually flushes after few retries and poll timeout does not expire. Pause/Resume may have some performance impact, like I read it flushes queued messages and stops fetching further records from broker, which is really not necessary if it's a momentary blip.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Sure, we can pause only after few retries of getting the needed buffer and that should help in doing all this in a momentary blip case.

paused = true;
Copy link
Member

Choose a reason for hiding this comment

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

Should kafka consumer be paused for all exception ?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

In all cases of exception, we are trying for ever to put in the buffer, right? that's why we are pausing in all cases of exceptions.

consumer.pause(consumer.assignment());
Copy link
Member

Choose a reason for hiding this comment

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

This is very critical logic. We should have a unit test to verify that we call pause() on these conditions.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Added new test case to test pause/resume

}
if (e instanceof SizeOverflowException) {
topicMetrics.getNumberOfBufferSizeOverflows().increment();
} else {
LOG.debug("Error while adding record to buffer, retrying ", e);
}
try {
Thread.sleep(100);
consumeRecords();
Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of conumeRecords() should we call consumer.poll() here directly ? and if it returns records then we assert and may be restart consumer because poll returning records means there is a bug somewhere. Calling consumeRecords() will re-enter this function if poll() returns messages.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

That's fair. I can just do poll() and assert on records returned.

} catch (Exception ex) {} // ignore the exception because it only means the thread slept for shorter time
}
}
if (paused) {
consumer.resume(consumer.assignment());
Copy link
Contributor

@hshardeesi hshardeesi Jan 27, 2024

Choose a reason for hiding this comment

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

I think there may be an issue here. We may have to call resume on exactly the same partitions that we paused. e.g. lets say consumer initially had 4 partitions that were paused. While paused, 2 got revoked and after reassignment we resumed only 2 assigned partitions. Now same 2 partitions are assigned back, will they remain paused? because we never called resume on them. It'll be good to test.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

If the partitions have moved out of the consumer, I do not think they will be in the paused state. I do not think we can resume the partitions that are currently not owned by a consumer. But will try to test it.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I have tested this scenario. When the partitions are assigned back, they are not paused. I think once the partitions are revoked, the partitions are removed from the consumer's assignment. When they are assigned back, they are assigned as any other new partition. There is no stale state from previous assignment.

paused = false;
}
}

private <T> void iterateRecordPartitions(ConsumerRecords<String, T> records, final AcknowledgementSet acknowledgementSet,
Expand Down Expand Up @@ -503,6 +514,9 @@ public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
LOG.info("Assigned partition {}", topicPartition);
ownedPartitionsEpoch.put(topicPartition, epoch);
}
if (paused) {
consumer.pause(consumer.assignment());
}
}
dumpTopicPartitionOffsets(partitions);
}
Expand All @@ -520,6 +534,9 @@ public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
ownedPartitionsEpoch.remove(topicPartition);
partitionCommitTrackerMap.remove(topicPartition.partition());
}
if (paused) {
Copy link
Contributor

Choose a reason for hiding this comment

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

is it really required in partition revocation?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Probably not. I thought about it before adding this code. I do not see any down side of doing it. Let me know if you can think of any side effects of this code.

consumer.pause(consumer.assignment());
}
}
}

Expand Down
Loading