-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-18618: Improve leader change handling of acknowledgements [1/N] #18672
base: trunk
Are you sure you want to change the base?
Conversation
I know this is still marked as a Draft, but how difficult will it be to add a test which tests the edge case which uncovered this? I believe I can follow the reasoning otherwise i.e. you only want to send the acknowledgements to the nodes from which the in-flight records originated. |
Not that difficult. Might be a follow-on PR to complete the testing, but we have two pieces of work in progress. First, we are developing tests using |
Awesome! Looking forward to seeing those 😊 |
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.
Looks mostly good to me. Left some minor comments.
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Acknowledgements.java
Show resolved
Hide resolved
* | ||
* @return the error code | ||
*/ | ||
public Errors getAcknowledgeErrorCode() { | ||
return acknowledgeErrorCode; | ||
public KafkaException getAcknowledgeException() { |
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.
We generally avoid get
prefix in methods so should it be as like below?
public KafkaException getAcknowledgeException() { | |
public KafkaException acknowledgeException() { |
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.
In public classes, yes, I agree. However, here there are many accessor methods already present and I don't think renaming them all improves things.
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Acknowledgements.java
Outdated
Show resolved
Hide resolved
this.acknowledgements = acknowledgements; | ||
} | ||
|
||
public int getNodeId() { |
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.
public int getNodeId() { | |
public int nodeId() { |
return nodeId; | ||
} | ||
|
||
public Acknowledgements getAcknowledgements() { |
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.
public Acknowledgements getAcknowledgements() { | |
public Acknowledgements acknowledgements() { |
/** | ||
* This class combines Acknowledgements with the id of the node to use for acknowledging. | ||
*/ | ||
public class NodeAcknowledgements { |
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.
nit: Seems this can be a record
class itself.
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.
Actually, I don't think that's allowed in the client. Records were introduced in Java 14 and can be used in the broker which builds with Java 17.
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.
Yeah you are right, client is not yet there.
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.
LGTM, just one query.
* <li> {@link WakeupException} if {@link KafkaShareConsumer#wakeup()} is called before or while this function is called | ||
* <li> {@link InterruptException} if the calling thread is interrupted before or while this function is called | ||
* <li> {@link KafkaException} for any other unrecoverable errors | ||
* </ul> | ||
* <p>Note that if the exception is a retriable exception, the acknowledgement could not be completed and the |
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.
nit: For non-retribale exceptions it will fetch anyways again so I expect we mean that even
for retriable exception, the fetch will happen again?
* <p>Note that if the exception is a retriable exception, the acknowledgement could not be completed and the | |
* <p>Note that if the exception is even a retriable exception, the acknowledgement could not be completed and the |
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.
I will reword.
metricsManager.recordAcknowledgementSent(acknowledgementsToSend.size()); | ||
fetchAcknowledgementsInFlight.put(tip, acknowledgementsToSend); | ||
Acknowledgements acknowledgementsToSend = null; | ||
Map<TopicIdPartition, Acknowledgements> nodeAcknowledgementsMap = fetchAcknowledgementsToSend.get(node.id()); |
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.
Query: Can there be ever some acknowledgments which can starve in the fetchAcknowledgementsToSend
when some nodeId is never received from broker?
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.
I think there will need to be a follow-on PR for this. In a way, you're correct, but I think to maintain the ordering of things, we need to fail some acknowledgements more aggressively from the poll loop. I had been avoiding that, but I think it's inevitable actually.
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.
Thank you.
if (acknowledgements != null) { | ||
acknowledgementsMapForNode.put(tip, acknowledgements); | ||
NodeAcknowledgements acknowledgements = acknowledgementsMap.get(tip); | ||
if ((acknowledgements != null) && (acknowledgements.nodeId() == node.id())) { |
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.
Here, should we add a null check for the actual Acknowledgements
class inside NodeAcknowledgements.
Although ideally as we already have a null check for NodeAcknowledgements, the case shouldn't ever arise. Just for it be consistent with other parts of the code.
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.
I will rename the variable acknowledgements
to nodeAcknowledgements
to make it a bit clearer to read. I don't believe that nodeAcknowledgements.acknowledgements()
can be null because the empty case is Acknowledgements.empty()
and the fetching code in the application thread only sends the acks to be processed if they are non-empty. I can put a null check in the constructor for NodeAcknowledgements
.
Map<TopicIdPartition, Acknowledgements> nodeAcksFromFetchMap = fetchAcknowledgementsToSend.get(nodeId); | ||
if (nodeAcksFromFetchMap != null) { | ||
nodeAcksFromFetchMap.forEach((tip, acks) -> { | ||
metricsManager.recordAcknowledgementSent(acks.size()); |
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.
Should we remove the mapping (tip, acks)
from nodeAcksFromFetchMap
(effectively removing them from fetchAcknowledgementsToSend
) so that we do not process these acknowledgements again in the next iteration.
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.
Yes, good point.
Acknowledgements acknowledgements = Acknowledgements.empty(); | ||
Map<TopicIdPartition, Acknowledgements> nodeAcksFromFetchMap = fetchAcknowledgementsToSend.get(nodeId); | ||
if (nodeAcksFromFetchMap != null) { | ||
Acknowledgements acksFromFetchMap = nodeAcksFromFetchMap.get(tip); |
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.
Should we do a nodeAcksFromFetchMap.remove(tip)
here? Although here as we are closing, we would not be using fetchAcknowledgementsToSend
again ideally.
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.
I'm not sure it matters because this is on the close path, but I think you're right that it's an improvement.
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.
@AndrewJSchofield Thanks for the PR!! Just had a couple of comments.
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.
Thanks for the PR! LGTM!
There is a problem in the share consumer where records fetched from node A might be acknowledged with node B, if node B used to be the leader of the topic-partition and A becomes unavailable.
Committer Checklist (excluded from commit message)