-
Notifications
You must be signed in to change notification settings - Fork 644
/
go_verifiable_producer.go
229 lines (191 loc) · 6.01 KB
/
go_verifiable_producer.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
/**
* Copyright 2016 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
// Apache Kafka kafkatest VerifiableProducer implemented in Go
package main
import (
"encoding/json"
"fmt"
"os"
"os/signal"
"strings"
"syscall"
"time"
"github.com/alecthomas/kingpin"
"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)
var (
verbosity = 1
sigs chan os.Signal
)
func send(name string, msg map[string]interface{}) {
if msg == nil {
msg = make(map[string]interface{})
}
msg["name"] = name
msg["_time"] = time.Now().Format("2006-01-02 15:04:05.000")
b, err := json.Marshal(msg)
if err != nil {
panic(err)
}
fmt.Println(string(b))
}
func partitionsToMap(partitions []kafka.TopicPartition) []map[string]interface{} {
parts := make([]map[string]interface{}, len(partitions))
for i, tp := range partitions {
parts[i] = map[string]interface{}{"topic": *tp.Topic, "partition": tp.Partition}
}
return parts
}
func sendPartitions(name string, partitions []kafka.TopicPartition) {
msg := make(map[string]interface{})
msg["partitions"] = partitionsToMap(partitions)
send(name, msg)
}
type commState struct {
maxMessages int // messages to send
msgCnt int // messages produced
deliveryCnt int // messages delivered
errCnt int // messages failed to deliver
valuePrefix string
throughput int
p *kafka.Producer
}
var state commState
// handle_dr handles delivery reports
// returns false when producer should terminate, else true to keep running.
func handleDr(m *kafka.Message) bool {
if verbosity >= 2 {
fmt.Fprintf(os.Stderr, "%% DR: %v:\n", m.TopicPartition)
}
if m.TopicPartition.Error != nil {
state.errCnt++
errmsg := make(map[string]interface{})
errmsg["message"] = m.TopicPartition.Error.Error()
errmsg["topic"] = *m.TopicPartition.Topic
errmsg["partition"] = m.TopicPartition.Partition
errmsg["key"] = (string)(m.Key)
errmsg["value"] = (string)(m.Value)
send("producer_send_error", errmsg)
} else {
state.deliveryCnt++
drmsg := make(map[string]interface{})
drmsg["topic"] = *m.TopicPartition.Topic
drmsg["partition"] = m.TopicPartition.Partition
drmsg["offset"] = m.TopicPartition.Offset
drmsg["key"] = (string)(m.Key)
drmsg["value"] = (string)(m.Value)
send("producer_send_success", drmsg)
}
if state.deliveryCnt+state.errCnt >= state.maxMessages {
// we're done
return false
}
return true
}
func runProducer(config *kafka.ConfigMap, topic string) {
p, err := kafka.NewProducer(config)
if err != nil {
fmt.Fprintf(os.Stderr, "Failed to create producer: %s\n", err)
os.Exit(1)
}
_, verstr := kafka.LibraryVersion()
fmt.Fprintf(os.Stderr, "%% Created Producer %v (%s)\n", p, verstr)
state.p = p
send("startup_complete", nil)
run := true
throttle := time.NewTicker(time.Second / (time.Duration)(state.throughput))
for run == true {
select {
case <-throttle.C:
// produce a message (async) on each throttler tick
value := fmt.Sprintf("%s%d", state.valuePrefix, state.msgCnt)
state.msgCnt++
err := p.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{
Topic: &topic,
Partition: kafka.PartitionAny},
Value: []byte(value)}, nil)
if err != nil {
fmt.Fprintf(os.Stderr, "%% Produce failed: %v\n", err)
state.errCnt++
}
if state.msgCnt == state.maxMessages {
// all messages sent, now wait for deliveries
throttle.Stop()
}
case sig := <-sigs:
fmt.Fprintf(os.Stderr, "%% Terminating on signal %v\n", sig)
run = false
case ev := <-p.Events():
switch e := ev.(type) {
case *kafka.Message:
run = handleDr(e)
case kafka.Error:
fmt.Fprintf(os.Stderr, "%% Error: %v\n", e)
run = false
default:
fmt.Fprintf(os.Stderr, "%% Unhandled event %T ignored: %v\n", e, e)
}
}
}
fmt.Fprintf(os.Stderr, "%% Closing, %d/%d messages delivered, %d failed\n", state.deliveryCnt, state.msgCnt, state.errCnt)
p.Close()
send("shutdown_complete", nil)
}
func main() {
sigs = make(chan os.Signal)
signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM)
// Default config
conf := kafka.ConfigMap{}
/* Required options */
topic := kingpin.Flag("topic", "Topic").Required().String()
brokers := kingpin.Flag("broker-list", "Bootstrap broker(s)").Required().String()
/* Optionals */
throughput := kingpin.Flag("throughput", "Msgs/s").Default("1000000").Int()
maxMessages := kingpin.Flag("max-messages", "Max message count").Default("1000000").Int()
valuePrefix := kingpin.Flag("value-prefix", "Payload value string prefix").Default("").String()
acks := kingpin.Flag("acks", "Required acks").Default("all").String()
configFile := kingpin.Flag("producer.config", "SerializerConfig file").File()
debug := kingpin.Flag("debug", "Debug flags").String()
xconf := kingpin.Flag("--property", "CSV separated key=value librdkafka configuration properties").Short('X').String()
kingpin.Parse()
conf["bootstrap.servers"] = *brokers
conf["acks"] = *acks
if len(*debug) > 0 {
conf["debug"] = *debug
}
if len(*xconf) > 0 {
for _, kv := range strings.Split(*xconf, ",") {
x := strings.Split(kv, "=")
if len(x) != 2 {
panic("-X expects a ,-separated list of confprop=val pairs")
}
conf[x[0]] = x[1]
}
}
fmt.Println("SerializerConfig: ", conf)
if *configFile != nil {
fmt.Fprintf(os.Stderr, "%% Ignoring config file %v\n", *configFile)
}
if len(*valuePrefix) > 0 {
state.valuePrefix = fmt.Sprintf("%s.", *valuePrefix)
} else {
state.valuePrefix = ""
}
state.throughput = *throughput
state.maxMessages = *maxMessages
runProducer((*kafka.ConfigMap)(&conf), *topic)
}