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

Adding producer latency to Message Delivery Report #1076

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
22 changes: 13 additions & 9 deletions kafka/message.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,14 +70,15 @@ func (t TimestampType) String() string {

// Message represents a Kafka message
type Message struct {
TopicPartition TopicPartition
Value []byte
Key []byte
Timestamp time.Time
TimestampType TimestampType
Opaque interface{}
Headers []Header
LeaderEpoch *int32 // Deprecated: LeaderEpoch or nil if not available. Use m.TopicPartition.LeaderEpoch instead.
TopicPartition TopicPartition
Value []byte
Key []byte
Timestamp time.Time
TimestampType TimestampType
Opaque interface{}
Headers []Header
LeaderEpoch *int32 // Deprecated: LeaderEpoch or nil if not available. Use m.TopicPartition.LeaderEpoch instead.
ProducerLatency *int64
}

// String returns a human readable representation of a Message.
Expand Down Expand Up @@ -167,13 +168,16 @@ func (h *handle) setupMessageFromC(msg *Message, cmsg *C.rd_kafka_message_t) {
msg.LeaderEpoch = &leaderEpoch
msg.TopicPartition.LeaderEpoch = &leaderEpoch
}
producerLatency := int64(C.rd_kafka_message_latency(cmsg))
if producerLatency >= 0 {
msg.ProducerLatency = &producerLatency
}
}

// newMessageFromC creates a new message object from a C rd_kafka_message_t
// NOTE: For use with Producer: does not set message timestamp fields.
func (h *handle) newMessageFromC(cmsg *C.rd_kafka_message_t) (msg *Message) {
msg = &Message{}

h.setupMessageFromC(msg, cmsg)

return msg
Expand Down
3 changes: 3 additions & 0 deletions kafka/producer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,9 @@ func TestProducerAPIs(t *testing.T) {
switch e := ev.(type) {
case *Message:
msgCnt++
if *e.ProducerLatency <= 0 {
t.Errorf("Producer Latency should be included in delivery reports, instead got %v", *e.ProducerLatency)
}
if (string)(e.Value) == "ProducerChannel" {
s := e.Opaque.(*string)
if s != &myOpq {
Expand Down