Skip to content

Commit 79fa6ef

Browse files
committed
Removed Produce() opaque arg in favour of Message.Opaque (issue confluentinc#4)
1 parent 6273e70 commit 79fa6ef

File tree

5 files changed

+33
-14
lines changed

5 files changed

+33
-14
lines changed

examples/producer_example/producer_example.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@ func main() {
4848
deliveryChan := make(chan kafka.Event)
4949

5050
value := "Hello Go!"
51-
err = p.Produce(&kafka.Message{TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: kafka.PartitionAny}, Value: []byte(value)}, deliveryChan, nil)
51+
err = p.Produce(&kafka.Message{TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: kafka.PartitionAny}, Value: []byte(value)}, deliveryChan)
5252

5353
e := <-deliveryChan
5454
m := e.(*kafka.Message)

kafka/producer.go

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ func (p *Producer) gethandle() *handle {
5656
return &p.handle
5757
}
5858

59-
func (p *Producer) produce(msg *Message, msgFlags int, deliveryChan chan Event, opaque interface{}) error {
59+
func (p *Producer) produce(msg *Message, msgFlags int, deliveryChan chan Event) error {
6060
crkt := p.handle.getRkt(*msg.TopicPartition.Topic)
6161

6262
var valp *byte
@@ -91,8 +91,8 @@ func (p *Producer) produce(msg *Message, msgFlags int, deliveryChan chan Event,
9191
// Since these cant be passed as opaque pointers to the C code,
9292
// due to cgo constraints, we add them to a per-producer map for lookup
9393
// when the C code triggers the callbacks or events.
94-
if deliveryChan != nil || opaque != nil {
95-
cgoid = p.handle.cgoPut(cgoDr{deliveryChan: deliveryChan, opaque: opaque})
94+
if deliveryChan != nil || msg.Opaque != nil {
95+
cgoid = p.handle.cgoPut(cgoDr{deliveryChan: deliveryChan, opaque: msg.Opaque})
9696
}
9797

9898
r := int(C.do_produce(crkt, C.int32_t(msg.TopicPartition.Partition),
@@ -116,8 +116,8 @@ func (p *Producer) produce(msg *Message, msgFlags int, deliveryChan chan Event,
116116
// The delivery report will be sent on the provided deliveryChan if specified,
117117
// or on the Producer object's Events channel if not.
118118
// Returns an error if message could not be enqueued.
119-
func (p *Producer) Produce(msg *Message, deliveryChan chan Event, opaque interface{}) error {
120-
return p.produce(msg, 0, deliveryChan, opaque)
119+
func (p *Producer) Produce(msg *Message, deliveryChan chan Event) error {
120+
return p.produce(msg, 0, deliveryChan)
121121
}
122122

123123
// Produce a batch of messages.
@@ -262,7 +262,7 @@ func NewProducer(conf *ConfigMap) (*Producer, error) {
262262
func channelProducer(p *Producer) {
263263

264264
for m := range p.ProduceChannel {
265-
err := p.produce(m, C.RD_KAFKA_MSG_F_BLOCK, nil, nil)
265+
err := p.produce(m, C.RD_KAFKA_MSG_F_BLOCK, nil)
266266
if err != nil {
267267
m.TopicPartition.Error = err
268268
p.Events <- m

kafka/producer_performance_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,7 @@ func BenchmarkProducerFunc(b *testing.B) {
149149
producerPerfTest(b, "Function producer (without DR)",
150150
0, false, false, false,
151151
func(p *Producer, m *Message, drChan chan Event) {
152-
err := p.Produce(m, drChan, nil)
152+
err := p.Produce(m, drChan)
153153
if err != nil {
154154
b.Errorf("Produce() failed: %v", err)
155155
}
@@ -160,7 +160,7 @@ func BenchmarkProducerFuncDR(b *testing.B) {
160160
producerPerfTest(b, "Function producer (with DR)",
161161
0, true, false, false,
162162
func(p *Producer, m *Message, drChan chan Event) {
163-
err := p.Produce(m, drChan, nil)
163+
err := p.Produce(m, drChan)
164164
if err != nil {
165165
b.Errorf("Produce() failed: %v", err)
166166
}

kafka/producer_test.go

Lines changed: 23 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -40,22 +40,25 @@ func TestProducerAPIs(t *testing.T) {
4040
// Produce with function, DR on passed drChan
4141
err = p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic1, Partition: 0},
4242
Value: []byte("Own drChan"), Key: []byte("This is my key")},
43-
drChan, nil)
43+
drChan)
4444
if err != nil {
4545
t.Errorf("Produce failed: %s", err)
4646
}
4747

4848
// Produce with function, use default DR channel (Events)
4949
err = p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0},
5050
Value: []byte("Events DR"), Key: []byte("This is my key")},
51-
nil, nil)
51+
nil)
5252
if err != nil {
5353
t.Errorf("Produce failed: %s", err)
5454
}
5555

56-
// Produce through ProducerChannel, uses default DR channel (Events)
56+
// Produce through ProducerChannel, uses default DR channel (Events),
57+
// pass Opaque object.
58+
myOpq := "My opaque"
5759
p.ProduceChannel <- &Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0},
58-
Value: []byte("ProducerChannel"), Key: []byte("This is my key")}
60+
Opaque: &myOpq,
61+
Value: []byte("ProducerChannel"), Key: []byte("This is my key")}
5962

6063
// Len() will not report messages on private delivery report chans (our drChan for example),
6164
// so expect at least 2 messages, not 3.
@@ -86,6 +89,22 @@ func TestProducerAPIs(t *testing.T) {
8689
switch e := ev.(type) {
8790
case *Message:
8891
msgCnt++
92+
if (string)(e.Value) == "ProducerChannel" {
93+
s := e.Opaque.(*string)
94+
if s != &myOpq {
95+
t.Errorf("Opaque should point to %v, not %v", &myOpq, s)
96+
}
97+
if *s != myOpq {
98+
t.Errorf("Opaque should be \"%s\", not \"%v\"",
99+
myOpq, *s)
100+
}
101+
t.Logf("Message \"%s\" with opaque \"%s\"\n",
102+
(string)(e.Value), *s)
103+
} else {
104+
if e.Opaque != nil {
105+
t.Errorf("Message opaque should be nil, not %v", e.Opaque)
106+
}
107+
}
89108
default:
90109
t.Logf("Ignored event %s", e)
91110
}

kafkatest/go_verifiable_producer/go_verifiable_producer.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -136,7 +136,7 @@ func runProducer(config *kafka.ConfigMap, topic string) {
136136
TopicPartition: kafka.TopicPartition{
137137
Topic: &topic,
138138
Partition: kafka.PartitionAny},
139-
Value: []byte(value)}, nil, nil)
139+
Value: []byte(value)}, nil)
140140
if err != nil {
141141
fmt.Fprintf(os.Stderr, "%% Produce failed: %v\n", err)
142142
state.errCnt++

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy