Skip to content

Commit 0c8f5b8

Browse files
committed
Add headers to go.delivery.report.fields (disabled by default)
Disabled due to the extra CGo calls required (costly) and the deserialization of C headers to Go Headers.
1 parent 8bb300e commit 0c8f5b8

File tree

7 files changed

+166
-75
lines changed

7 files changed

+166
-75
lines changed

CHANGELOG.md

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,19 @@
22

33
## v1.7.0
44

5+
confluent-kafka-go is based on librdkafka v1.7.0, see the
6+
[librdkafka release notes](https://github.com/edenhill/librdkafka/releases/tag/v1.7.0)
7+
for a complete list of changes, enhancements, fixes and upgrade considerations.
8+
9+
### Enhancements
10+
11+
* The produced message headers are now available in the delivery report
12+
`Message.Headers` if the Producer's `go.delivery.report.fields`
13+
configuration property is set to include `headers`, e.g.:
14+
`"go.delivery.report.fields": "key,value,headers"`
15+
This comes at a performance cost and are thus disabled by default.
16+
17+
518
### Fixes
619

720
* AdminClient.CreateTopics() previously did not accept default value(-1) of

kafka/event.go

Lines changed: 37 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -28,24 +28,32 @@ import (
2828
#include "glue_rdkafka.h"
2929
3030
31-
#ifdef RD_KAFKA_V_HEADERS
32-
void chdrs_to_tmphdrs (rd_kafka_headers_t *chdrs, tmphdr_t *tmphdrs) {
33-
size_t i = 0;
34-
const char *name;
35-
const void *val;
36-
size_t size;
37-
38-
while (!rd_kafka_header_get_all(chdrs, i,
39-
&tmphdrs[i].key,
40-
&tmphdrs[i].val,
41-
(size_t *)&tmphdrs[i].size))
42-
i++;
31+
void chdrs_to_tmphdrs (glue_msg_t *gMsg) {
32+
size_t i = 0;
33+
const char *name;
34+
const void *val;
35+
size_t size;
36+
rd_kafka_headers_t *chdrs;
37+
38+
if (rd_kafka_message_headers(gMsg->msg, &chdrs)) {
39+
gMsg->tmphdrs = NULL;
40+
gMsg->tmphdrsCnt = 0;
41+
return;
42+
}
43+
44+
gMsg->tmphdrsCnt = rd_kafka_header_cnt(chdrs);
45+
gMsg->tmphdrs = malloc(sizeof(*gMsg->tmphdrs) * gMsg->tmphdrsCnt);
46+
47+
while (!rd_kafka_header_get_all(chdrs, i,
48+
&gMsg->tmphdrs[i].key,
49+
&gMsg->tmphdrs[i].val,
50+
(size_t *)&gMsg->tmphdrs[i].size))
51+
i++;
4352
}
44-
#endif
4553
4654
rd_kafka_event_t *_rk_queue_poll (rd_kafka_queue_t *rkq, int timeoutMs,
4755
rd_kafka_event_type_t *evtype,
48-
fetched_c_msg_t *fcMsg,
56+
glue_msg_t *gMsg,
4957
rd_kafka_event_t *prev_rkev) {
5058
rd_kafka_event_t *rkev;
5159
@@ -56,31 +64,22 @@ rd_kafka_event_t *_rk_queue_poll (rd_kafka_queue_t *rkq, int timeoutMs,
5664
*evtype = rd_kafka_event_type(rkev);
5765
5866
if (*evtype == RD_KAFKA_EVENT_FETCH) {
59-
#ifdef RD_KAFKA_V_HEADERS
60-
rd_kafka_headers_t *hdrs;
61-
#endif
62-
63-
fcMsg->msg = (rd_kafka_message_t *)rd_kafka_event_message_next(rkev);
64-
fcMsg->ts = rd_kafka_message_timestamp(fcMsg->msg, &fcMsg->tstype);
65-
66-
#ifdef RD_KAFKA_V_HEADERS
67-
if (!rd_kafka_message_headers(fcMsg->msg, &hdrs)) {
68-
fcMsg->tmphdrsCnt = rd_kafka_header_cnt(hdrs);
69-
fcMsg->tmphdrs = malloc(sizeof(*fcMsg->tmphdrs) * fcMsg->tmphdrsCnt);
70-
chdrs_to_tmphdrs(hdrs, fcMsg->tmphdrs);
71-
} else {
72-
#else
73-
if (1) {
74-
#endif
75-
fcMsg->tmphdrs = NULL;
76-
fcMsg->tmphdrsCnt = 0;
77-
}
67+
gMsg->msg = (rd_kafka_message_t *)rd_kafka_event_message_next(rkev);
68+
gMsg->ts = rd_kafka_message_timestamp(gMsg->msg, &gMsg->tstype);
69+
70+
if (gMsg->want_hdrs)
71+
chdrs_to_tmphdrs(gMsg);
7872
}
73+
7974
return rkev;
8075
}
8176
*/
8277
import "C"
8378

79+
func chdrsToTmphdrs(gMsg *C.glue_msg_t) {
80+
C.chdrs_to_tmphdrs(gMsg)
81+
}
82+
8483
// Event generic interface
8584
type Event interface {
8685
// String returns a human-readable representation of the event
@@ -164,8 +163,9 @@ func (h *handle) eventPoll(channel chan Event, timeoutMs int, maxEvents int, ter
164163
out:
165164
for evcnt := 0; evcnt < maxEvents; evcnt++ {
166165
var evtype C.rd_kafka_event_type_t
167-
var fcMsg C.fetched_c_msg_t
168-
rkev := C._rk_queue_poll(h.rkq, C.int(timeoutMs), &evtype, &fcMsg, prevRkev)
166+
var gMsg C.glue_msg_t
167+
gMsg.want_hdrs = C.int8_t(bool2cint(h.msgFields.Headers))
168+
rkev := C._rk_queue_poll(h.rkq, C.int(timeoutMs), &evtype, &gMsg, prevRkev)
169169
prevRkev = rkev
170170
timeoutMs = 0
171171

@@ -174,8 +174,8 @@ out:
174174
switch evtype {
175175
case C.RD_KAFKA_EVENT_FETCH:
176176
// Consumer fetch event, new message.
177-
// Extracted into temporary fcMsg for optimization
178-
retval = h.newMessageFromFcMsg(&fcMsg)
177+
// Extracted into temporary gMsg for optimization
178+
retval = h.newMessageFromGlueMsg(&gMsg)
179179

180180
case C.RD_KAFKA_EVENT_REBALANCE:
181181
// Consumer rebalance event

kafka/glue_rdkafka.h

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -34,13 +34,15 @@ typedef struct tmphdr_s {
3434

3535

3636
/**
37-
* Represents a fetched C message, with all extra fields extracted
38-
* to struct fields.
37+
* @struct This is a glue struct used by the C code in this client to
38+
* effectively map fields from a librdkafka rd_kafka_message_t
39+
* to something usable in Go with as few CGo calls as possible.
3940
*/
40-
typedef struct fetched_c_msg {
41+
typedef struct glue_msg_s {
4142
rd_kafka_message_t *msg;
4243
rd_kafka_timestamp_type_t tstype;
43-
int64_t ts;
44+
int64_t ts;
4445
tmphdr_t *tmphdrs;
4546
size_t tmphdrsCnt;
46-
} fetched_c_msg_t;
47+
int8_t want_hdrs; /**< If true, copy headers */
48+
} glue_msg_t;

kafka/handle.go

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -114,7 +114,7 @@ type handle struct {
114114
// Forward delivery reports on Producer.Events channel
115115
fwdDr bool
116116

117-
// Enabled fields for delivery reports
117+
// Enabled message fields for delivery reports and consumed messages.
118118
msgFields *messageFields
119119

120120
//
@@ -328,24 +328,27 @@ func (h *handle) setOAuthBearerTokenFailure(errstr string) error {
328328
return newError(cErr)
329329
}
330330

331-
// messageFields controls which fields are made available for producer delivery reports & incoming messages
331+
// messageFields controls which fields are made available for producer delivery reports & consumed messages.
332332
// true values indicate that the field should be included
333333
type messageFields struct {
334-
Key bool
335-
Value bool
334+
Key bool
335+
Value bool
336+
Headers bool
336337
}
337338

338339
// disableAll disable all fields
339340
func (mf *messageFields) disableAll() {
340341
mf.Key = false
341342
mf.Value = false
343+
mf.Headers = false
342344
}
343345

344346
// newMessageFields returns a new messageFields with all fields enabled
345347
func newMessageFields() *messageFields {
346348
return &messageFields{
347-
Key: true,
348-
Value: true,
349+
Key: true,
350+
Value: true,
351+
Headers: true,
349352
}
350353
}
351354

@@ -365,6 +368,8 @@ func newMessageFieldsFrom(v ConfigValue) (*messageFields, error) {
365368
msgFields.Key = true
366369
case "value":
367370
msgFields.Value = true
371+
case "headers":
372+
msgFields.Headers = true
368373
default:
369374
return nil, fmt.Errorf("unknown message field: %s", value)
370375
}

kafka/message.go

Lines changed: 33 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -100,30 +100,37 @@ func (h *handle) getRktFromMessage(msg *Message) (crkt *C.rd_kafka_topic_t) {
100100
return h.getRkt(*msg.TopicPartition.Topic)
101101
}
102102

103-
func (h *handle) newMessageFromFcMsg(fcMsg *C.fetched_c_msg_t) (msg *Message) {
103+
// setupHeadersFromGlueMsg converts the C tmp headers in gMsg to
104+
// Go Headers in msg.
105+
// gMsg.tmphdrs will be freed.
106+
func setupHeadersFromGlueMsg(msg *Message, gMsg *C.glue_msg_t) {
107+
msg.Headers = make([]Header, gMsg.tmphdrsCnt)
108+
for n := range msg.Headers {
109+
tmphdr := (*[1 << 30]C.tmphdr_t)(unsafe.Pointer(gMsg.tmphdrs))[n]
110+
msg.Headers[n].Key = C.GoString(tmphdr.key)
111+
if tmphdr.val != nil {
112+
msg.Headers[n].Value = C.GoBytes(unsafe.Pointer(tmphdr.val), C.int(tmphdr.size))
113+
} else {
114+
msg.Headers[n].Value = nil
115+
}
116+
}
117+
C.free(unsafe.Pointer(gMsg.tmphdrs))
118+
}
119+
120+
func (h *handle) newMessageFromGlueMsg(gMsg *C.glue_msg_t) (msg *Message) {
104121
msg = &Message{}
105122

106-
if fcMsg.ts != -1 {
107-
ts := int64(fcMsg.ts)
108-
msg.TimestampType = TimestampType(fcMsg.tstype)
123+
if gMsg.ts != -1 {
124+
ts := int64(gMsg.ts)
125+
msg.TimestampType = TimestampType(gMsg.tstype)
109126
msg.Timestamp = time.Unix(ts/1000, (ts%1000)*1000000)
110127
}
111128

112-
if fcMsg.tmphdrsCnt > 0 {
113-
msg.Headers = make([]Header, fcMsg.tmphdrsCnt)
114-
for n := range msg.Headers {
115-
tmphdr := (*[1 << 30]C.tmphdr_t)(unsafe.Pointer(fcMsg.tmphdrs))[n]
116-
msg.Headers[n].Key = C.GoString(tmphdr.key)
117-
if tmphdr.val != nil {
118-
msg.Headers[n].Value = C.GoBytes(unsafe.Pointer(tmphdr.val), C.int(tmphdr.size))
119-
} else {
120-
msg.Headers[n].Value = nil
121-
}
122-
}
123-
C.free(unsafe.Pointer(fcMsg.tmphdrs))
129+
if gMsg.tmphdrsCnt > 0 {
130+
setupHeadersFromGlueMsg(msg, gMsg)
124131
}
125132

126-
h.setupMessageFromC(msg, fcMsg.msg)
133+
h.setupMessageFromC(msg, gMsg.msg)
127134

128135
return msg
129136
}
@@ -141,6 +148,15 @@ func (h *handle) setupMessageFromC(msg *Message, cmsg *C.rd_kafka_message_t) {
141148
if cmsg.key != nil && h.msgFields.Key {
142149
msg.Key = C.GoBytes(unsafe.Pointer(cmsg.key), C.int(cmsg.key_len))
143150
}
151+
if h.msgFields.Headers {
152+
var gMsg C.glue_msg_t
153+
gMsg.msg = cmsg
154+
gMsg.want_hdrs = C.int8_t(1)
155+
chdrsToTmphdrs(&gMsg)
156+
if gMsg.tmphdrsCnt > 0 {
157+
setupHeadersFromGlueMsg(msg, &gMsg)
158+
}
159+
}
144160
msg.TopicPartition.Offset = Offset(cmsg.offset)
145161
if cmsg.err != 0 {
146162
msg.TopicPartition.Error = newError(cmsg.err)

kafka/producer.go

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -417,14 +417,15 @@ func (p *Producer) Purge(flags int) error {
417417
//
418418
// conf is a *ConfigMap with standard librdkafka configuration properties.
419419
//
420-
// Supported special configuration properties:
420+
// Supported special configuration properties (type, default):
421421
// go.batch.producer (bool, false) - EXPERIMENTAL: Enable batch producer (for increased performance).
422422
// These batches do not relate to Kafka message batches in any way.
423423
// Note: timestamps and headers are not supported with this interface.
424424
// go.delivery.reports (bool, true) - Forward per-message delivery reports to the
425425
// Events() channel.
426-
// go.delivery.report.fields (string, all) - Comma separated list of fields to enable for delivery reports.
427-
// Allowed values: all, none (or empty string), key, value
426+
// go.delivery.report.fields (string, "key,value") - Comma separated list of fields to enable for delivery reports.
427+
// Allowed values: all, none (or empty string), key, value, headers
428+
// Warning: There is a performance penalty to include headers in the delivery report.
428429
// go.events.channel.size (int, 1000000) - Events().
429430
// go.produce.channel.size (int, 1000000) - ProduceChannel() buffer size (in number of messages)
430431
// go.logs.channel.enable (bool, false) - Forward log to Logs() channel.
@@ -463,16 +464,15 @@ func NewProducer(conf *ConfigMap) (*Producer, error) {
463464
}
464465
p.handle.fwdDr = v.(bool)
465466

466-
v, err = confCopy.extract("go.delivery.report.fields", "all")
467+
v, err = confCopy.extract("go.delivery.report.fields", "key,value")
467468
if err != nil {
468469
return nil, err
469470
}
470471

471-
msgFields, err := newMessageFieldsFrom(v)
472+
p.handle.msgFields, err = newMessageFieldsFrom(v)
472473
if err != nil {
473474
return nil, err
474475
}
475-
p.handle.msgFields = msgFields
476476

477477
v, err = confCopy.extract("go.events.channel.size", 1000000)
478478
if err != nil {

0 commit comments

Comments
 (0)