Skip to content

Commit 882f4b1

Browse files
committed
Fix 'go vet' warnings for cgomap
1 parent b602975 commit 882f4b1

File tree

4 files changed

+18
-10
lines changed

4 files changed

+18
-10
lines changed

kafka/consumer.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -295,7 +295,7 @@ func NewConsumer(conf *ConfigMap) (*Consumer, error) {
295295

296296
c.handle.c = c
297297
c.handle.setup()
298-
c.handle.cgomap = make(map[uintptr]cgoif)
298+
c.handle.cgomap = make(map[int]cgoif)
299299
c.handle.rkq = C.rd_kafka_queue_get_consumer(c.handle.rk)
300300
if c.handle.rkq == nil {
301301
// no cgrp (no group.id configured), revert to main queue.

kafka/event.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ out:
203203

204204
if rkmessage._private != nil {
205205
// Find cgoif by id
206-
cg, found := h.cgo_get((uintptr)(rkmessage._private))
206+
cg, found := h.cgo_get((int)((uintptr)(rkmessage._private)))
207207
if found {
208208
cdr := cg.(cgo_dr)
209209

kafka/handle.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ type handle struct {
5555
// Maps C callbacks based on cgoid back to its Go object
5656
cgo_lock sync.Mutex
5757
cgoid_next uintptr
58-
cgomap map[uintptr]cgoif
58+
cgomap map[int]cgoif
5959

6060
//
6161
// producer
@@ -176,23 +176,23 @@ type cgo_dr struct {
176176
// unique id for the added entry.
177177
// Thread-safe.
178178
// FIXME: the uniquity of the id is questionable over time.
179-
func (h *handle) cgo_put(cg cgoif) (cgoid uintptr) {
179+
func (h *handle) cgo_put(cg cgoif) (cgoid int) {
180180
h.cgo_lock.Lock()
181181
defer h.cgo_lock.Unlock()
182182

183183
h.cgoid_next += 1
184184
if h.cgoid_next == 0 {
185185
h.cgoid_next += 1
186186
}
187-
cgoid = h.cgoid_next
187+
cgoid = (int)(h.cgoid_next)
188188
h.cgomap[cgoid] = cg
189189
return cgoid
190190
}
191191

192192
// cgo_get looks up cgoid in the cgo map, deletes the reference from the map
193193
// and returns the object, if found. Else returns nil, false.
194194
// Thread-safe.
195-
func (h *handle) cgo_get(cgoid uintptr) (cg cgoif, found bool) {
195+
func (h *handle) cgo_get(cgoid int) (cg cgoif, found bool) {
196196
if cgoid == 0 {
197197
return nil, false
198198
}

kafka/producer.go

Lines changed: 12 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,13 @@ import (
2626
/*
2727
#include <stdlib.h>
2828
#include <librdkafka/rdkafka.h>
29+
30+
rd_kafka_resp_err_t do_produce (rd_kafka_topic_t *rkt, int32_t partition,
31+
int msg_flags,
32+
void *val, size_t val_len, void *key, size_t key_len, uintptr_t cgoid) {
33+
return rd_kafka_produce(rkt, partition, msg_flags, val, val_len, key, key_len,
34+
(void *)cgoid);
35+
}
2936
*/
3037
import "C"
3138

@@ -81,7 +88,7 @@ func (p *Producer) produce(msg *Message, msg_flags int, delivery_chan chan Event
8188
}
8289
}
8390

84-
var cgoid uintptr
91+
var cgoid int
8592

8693
// Per-message state that needs to be retained through the C code:
8794
// delivery channel (if specified)
@@ -93,10 +100,11 @@ func (p *Producer) produce(msg *Message, msg_flags int, delivery_chan chan Event
93100
cgoid = p.handle.cgo_put(cgo_dr{delivery_chan: delivery_chan, opaque: opaque})
94101
}
95102

96-
r := int(C.rd_kafka_produce(c_rkt, C.int32_t(msg.TopicPartition.Partition),
103+
r := int(C.do_produce(c_rkt, C.int32_t(msg.TopicPartition.Partition),
97104
C.int(msg_flags)|C.RD_KAFKA_MSG_F_COPY,
98105
unsafe.Pointer(valp), C.size_t(val_len),
99-
unsafe.Pointer(keyp), C.size_t(key_len), unsafe.Pointer(cgoid)))
106+
unsafe.Pointer(keyp), C.size_t(key_len),
107+
(C.uintptr_t)(cgoid)))
100108
if r == -1 {
101109
if cgoid != 0 {
102110
p.handle.cgo_get(cgoid)
@@ -232,7 +240,7 @@ func NewProducer(conf *ConfigMap) (*Producer, error) {
232240
p.handle.p = p
233241
p.handle.setup()
234242
p.handle.rkq = C.rd_kafka_queue_get_main(p.handle.rk)
235-
p.handle.cgomap = make(map[uintptr]cgoif)
243+
p.handle.cgomap = make(map[int]cgoif)
236244
p.Events = make(chan Event, 1000000)
237245
p.ProduceChannel = make(chan *Message, produce_channel_size)
238246
p.poller_term_chan = make(chan bool)

0 commit comments

Comments
 (0)