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

context.Done() may never reach if waiting on r.incoming <- msgErr #936

Merged
merged 1 commit into from
Sep 13, 2023
Merged
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
12 changes: 10 additions & 2 deletions protocol/kafka_sarama/v2/receiver.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,15 +66,23 @@ func (r *Receiver) ConsumeClaim(session sarama.ConsumerGroupSession, claim saram
return nil
}
m := NewMessageFromConsumerMessage(msg)

r.incoming <- msgErr{
msgErrObj := msgErr{
msg: binding.WithFinish(m, func(err error) {
if protocol.IsACK(err) {
session.MarkMessage(msg, "")
}
}),
}

// Need to use select clause here, otherwise r.incoming <- msgErrObj can become a blocking operation,
// resulting in never reaching outside block's case <-session.Context().Done()
select {
case r.incoming <- msgErrObj:
// do nothing
case <-session.Context().Done():
Copy link
Member

Choose a reason for hiding this comment

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

spot on! usually we'd return session.Context.Err() to let the caller know why the function returned. But this would break the existing behavior (returning nil). A user could always use context.Cause(ctx), but that might not be intuitive from the nil behavior.

Anyways, IMHO with this change you can remove the code in line 95?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Code on line 95 is still required. If code on line no. 64 case msg, ok := <-claim.Messages(): doesn't get hit, that code on line no. 95 would exit the flow.

Copy link
Member

Choose a reason for hiding this comment

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

True, was under the assumption that ok would be false due to propagated channel somewhere into Sarama...but seems that's not the case.

return nil
}

// Should return when `session.Context()` is done.
// If not, will raise `ErrRebalanceInProgress` or `read tcp <ip>:<port>: i/o timeout` when kafka rebalance. see:
// https://github.com/Shopify/sarama/issues/1192
Expand Down