forked from fgeller/kt
-
Notifications
You must be signed in to change notification settings - Fork 0
/
group.go
516 lines (425 loc) · 13.1 KB
/
group.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
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
package main
import (
"flag"
"fmt"
"log"
"os"
"os/user"
"regexp"
"sort"
"strconv"
"strings"
"sync"
"github.com/IBM/sarama"
dps "github.com/markusmobius/go-dateparser"
)
type groupCmd struct {
baseCmd
brokers []string
auth authConfig
group string
filterGroups *regexp.Regexp
filterTopics *regexp.Regexp
topic string
partitions []int32
reset int64
resetTime bool
pretty bool
version sarama.KafkaVersion
offsets bool
client sarama.Client
}
type group struct {
Name string `json:"name"`
Topic string `json:"topic,omitempty"`
Offsets []groupOffset `json:"offsets,omitempty"`
}
type groupOffset struct {
Partition int32 `json:"partition"`
Offset *int64 `json:"offset"`
Lag *int64 `json:"lag"`
}
const (
allPartitionsHuman = "all"
resetNotSpecified = -23
)
func (cmd *groupCmd) run(args []string) {
var err error
cmd.parseArgs(args)
if cmd.verbose {
sarama.Logger = log.New(os.Stderr, "", log.LstdFlags)
}
if cmd.client, err = sarama.NewClient(cmd.brokers, cmd.saramaConfig()); err != nil {
failf("failed to create client err=%v", err)
}
brokers := cmd.client.Brokers()
cmd.infof("found %v brokers\n", len(brokers))
groups := []string{cmd.group}
if cmd.group == "" {
groups = []string{}
for _, g := range cmd.findGroups(brokers) {
if cmd.filterGroups.MatchString(g) {
groups = append(groups, g)
}
}
}
cmd.infof("found %v groups\n", len(groups))
topics := []string{cmd.topic}
if cmd.topic == "" {
topics = []string{}
for _, t := range cmd.fetchTopics() {
if cmd.filterTopics.MatchString(t) {
topics = append(topics, t)
}
}
}
cmd.infof("found %v topics\n", len(topics))
out := make(chan printContext)
go print(out, cmd.pretty)
if !cmd.offsets {
for i, grp := range groups {
ctx := printContext{output: group{Name: grp}, done: make(chan struct{})}
out <- ctx
<-ctx.done
cmd.infof("%v/%v\n", i+1, len(groups))
}
return
}
topicPartitions := map[string][]int32{}
for _, topic := range topics {
parts := cmd.partitions
if len(parts) == 0 {
parts = cmd.fetchPartitions(topic)
cmd.infof("found partitions=%v for topic=%v\n", parts, topic)
}
topicPartitions[topic] = parts
}
wg := &sync.WaitGroup{}
wg.Add(len(groups) * len(topics))
for _, grp := range groups {
for top, parts := range topicPartitions {
go func(grp, topic string, partitions []int32) {
cmd.printGroupTopicOffset(out, grp, topic, partitions)
wg.Done()
}(grp, top, parts)
}
}
wg.Wait()
}
func (cmd *groupCmd) printGroupTopicOffset(out chan printContext, grp, top string, parts []int32) {
target := group{Name: grp, Topic: top, Offsets: make([]groupOffset, 0, len(parts))}
results := make(chan groupOffset)
done := make(chan struct{})
wg := &sync.WaitGroup{}
wg.Add(len(parts))
for _, part := range parts {
go cmd.fetchGroupOffset(wg, grp, top, part, results)
}
go func() { wg.Wait(); close(done) }()
awaitGroupOffsets:
for {
select {
case res := <-results:
target.Offsets = append(target.Offsets, res)
case <-done:
break awaitGroupOffsets
}
}
if len(target.Offsets) > 0 {
sort.Slice(target.Offsets, func(i, j int) bool {
return target.Offsets[j].Partition > target.Offsets[i].Partition
})
ctx := printContext{output: target, done: make(chan struct{})}
out <- ctx
<-ctx.done
}
}
func (cmd *groupCmd) resolveOffset(top string, part int32, time int64) int64 {
resolvedOff, err := cmd.client.GetOffset(top, part, time)
if err != nil {
failf("failed to get offset to reset to for partition=%d err=%v", part, err)
}
cmd.infof("resolved offset %v for topic=%s partition=%d to %v\n", time, top, part, resolvedOff)
return resolvedOff
}
func (cmd *groupCmd) fetchGroupOffset(wg *sync.WaitGroup, grp, top string, part int32, results chan<- groupOffset) {
defer wg.Done()
cmd.infof("fetching offset information for group=%v topic=%v partition=%v\n", grp, top, part)
offsetManager, err := sarama.NewOffsetManagerFromClient(grp, cmd.client)
if err != nil {
failf("failed to create client err=%v", err)
}
defer logClose("offset manager", offsetManager)
pom, err := offsetManager.ManagePartition(top, part)
if err != nil {
failf("failed to manage partition group=%s topic=%s partition=%d err=%v", grp, top, part, err)
}
defer logClose("partition offset manager", pom)
specialOffset := cmd.resetTime || cmd.reset == sarama.OffsetNewest || cmd.reset == sarama.OffsetOldest
groupOff, _ := pom.NextOffset()
if cmd.reset >= 0 || specialOffset {
resolvedOff := cmd.reset
if specialOffset {
resolvedOff = cmd.resolveOffset(top, part, cmd.reset)
}
if resolvedOff > groupOff {
pom.MarkOffset(resolvedOff, "")
} else {
pom.ResetOffset(resolvedOff, "")
}
groupOff = resolvedOff
}
partOff := cmd.resolveOffset(top, part, sarama.OffsetNewest)
lag := partOff - groupOff
results <- groupOffset{Partition: part, Offset: &groupOff, Lag: &lag}
}
func (cmd *groupCmd) fetchTopics() []string {
tps, err := cmd.client.Topics()
if err != nil {
failf("failed to read topics err=%v", err)
}
return tps
}
func (cmd *groupCmd) fetchPartitions(top string) []int32 {
ps, err := cmd.client.Partitions(top)
if err != nil {
failf("failed to read partitions for topic=%s err=%v", top, err)
}
return ps
}
type findGroupResult struct {
done bool
group string
}
func (cmd *groupCmd) findGroups(brokers []*sarama.Broker) []string {
var (
doneCount int
groups = []string{}
results = make(chan findGroupResult)
errs = make(chan error)
)
for _, broker := range brokers {
go cmd.findGroupsOnBroker(broker, results, errs)
}
awaitGroups:
for {
if doneCount == len(brokers) {
return groups
}
select {
case err := <-errs:
failf("failed to find groups err=%v", err)
case res := <-results:
if res.done {
doneCount++
continue awaitGroups
}
groups = append(groups, res.group)
}
}
}
func (cmd *groupCmd) findGroupsOnBroker(broker *sarama.Broker, results chan findGroupResult, errs chan error) {
var (
err error
resp *sarama.ListGroupsResponse
)
if err = cmd.connect(broker); err != nil {
errs <- fmt.Errorf("failed to connect to broker %#v err=%s\n", broker.Addr(), err)
}
if resp, err = broker.ListGroups(&sarama.ListGroupsRequest{}); err != nil {
errs <- fmt.Errorf("failed to list brokers on %#v err=%v", broker.Addr(), err)
}
if resp.Err != sarama.ErrNoError {
errs <- fmt.Errorf("failed to list brokers on %#v err=%v", broker.Addr(), resp.Err)
}
for name := range resp.Groups {
results <- findGroupResult{group: name}
}
results <- findGroupResult{done: true}
}
func (cmd *groupCmd) connect(broker *sarama.Broker) error {
if ok, _ := broker.Connected(); ok {
return nil
}
if err := broker.Open(cmd.saramaConfig()); err != nil {
return err
}
connected, err := broker.Connected()
if err != nil {
return err
}
if !connected {
return fmt.Errorf("failed to connect broker %#v", broker.Addr())
}
return nil
}
func (cmd *groupCmd) saramaConfig() *sarama.Config {
var (
err error
usr *user.User
cfg = sarama.NewConfig()
)
cfg.Version = cmd.version
if usr, err = user.Current(); err != nil {
cmd.infof("Failed to read current user err=%v", err)
}
cfg.ClientID = "kt-group-" + sanitizeUsername(usr.Username)
cmd.infof("sarama client configuration %#v\n", cfg)
if err = setupAuth(cmd.auth, cfg); err != nil {
failf("failed to setup auth err=%v", err)
}
return cfg
}
func (cmd *groupCmd) failStartup(msg string) {
warnf(msg)
failf("use \"kt group -help\" for more information")
}
func (cmd *groupCmd) parseArgs(as []string) {
var (
err error
args = cmd.parseFlags(as)
)
envTopic := os.Getenv(ENV_TOPIC)
if args.topic == "" {
args.topic = envTopic
}
cmd.topic = args.topic
cmd.group = args.group
cmd.verbose = args.verbose
cmd.pretty = args.pretty
cmd.offsets = args.offsets
cmd.version, err = chooseKafkaVersion(args.version, os.Getenv(ENV_KAFKA_VERSION))
if err != nil {
failf("failed to read kafka version err=%v", err)
}
readAuthFile(args.auth, os.Getenv(ENV_AUTH), &cmd.auth)
switch args.partitions {
case "", "all":
cmd.partitions = []int32{}
default:
pss := strings.Split(args.partitions, ",")
for _, ps := range pss {
p, err := strconv.ParseInt(ps, 10, 32)
if err != nil {
failf("partition id invalid err=%v", err)
}
cmd.partitions = append(cmd.partitions, int32(p))
}
}
if cmd.partitions == nil {
failf(`failed to interpret partitions flag %#v. Should be a comma separated list of partitions or "all".`, args.partitions)
}
if cmd.filterGroups, err = regexp.Compile(args.filterGroups); err != nil {
failf("groups filter regexp invalid err=%v", err)
}
if cmd.filterTopics, err = regexp.Compile(args.filterTopics); err != nil {
failf("topics filter regexp invalid err=%v", err)
}
if args.reset != "" && (args.topic == "" || args.group == "") {
failf("group and topic are required to reset offsets.")
}
cmd.resetTime = false
switch args.reset {
case "newest":
cmd.reset = sarama.OffsetNewest
case "oldest":
cmd.reset = sarama.OffsetOldest
case "":
// optional flag
cmd.reset = resetNotSpecified
default:
cmd.reset, err = strconv.ParseInt(args.reset, 10, 64)
if err != nil {
var dt, derr = dps.Parse(nil, args.reset)
if derr == nil {
err = nil
cmd.reset = dt.Time.UnixMilli()
cmd.resetTime = true
} else {
err = derr
}
}
if err != nil {
warnf("failed to parse set %#v err=%v", args.reset, err)
cmd.failStartup(fmt.Sprintf(`set value %#v not valid. either "newest", "oldest", a time, or a specific offset expected. See https://github.com/markusmobius/go-dateparser for supported time formats. `, args.reset))
}
}
envBrokers := os.Getenv(ENV_BROKERS)
if args.brokers == "" {
if envBrokers != "" {
args.brokers = envBrokers
} else {
args.brokers = "localhost:9092"
}
}
cmd.brokers = strings.Split(args.brokers, ",")
for i, b := range cmd.brokers {
if !strings.Contains(b, ":") {
cmd.brokers[i] = b + ":9092"
}
}
}
type groupArgs struct {
topic string
brokers string
auth string
partitions string
group string
filterGroups string
filterTopics string
reset string
resetTime bool
verbose bool
pretty bool
version string
offsets bool
}
func (cmd *groupCmd) parseFlags(as []string) groupArgs {
var args groupArgs
flags := flag.NewFlagSet("group", flag.ContinueOnError)
flags.StringVar(&args.topic, "topic", "", "Topic to consume (required).")
flags.StringVar(&args.brokers, "brokers", "", "Comma separated list of brokers. Port defaults to 9092 when omitted (defaults to localhost:9092).")
flags.StringVar(&args.auth, "auth", "", fmt.Sprintf("Path to auth configuration file, can also be set via %s env variable", ENV_AUTH))
flags.StringVar(&args.group, "group", "", "Consumer group name.")
flags.StringVar(&args.filterGroups, "filter-groups", "", "Regex to filter groups.")
flags.StringVar(&args.filterTopics, "filter-topics", "", "Regex to filter topics.")
flags.StringVar(&args.reset, "reset", "", "Target offset to reset for consumer group (\"newest\", \"oldest\", a time, or specific offset)")
flags.BoolVar(&args.verbose, "verbose", false, "More verbose logging to stderr.")
flags.BoolVar(&args.pretty, "pretty", true, "Control output pretty printing.")
flags.StringVar(&args.version, "version", "", "Kafka protocol version")
flags.StringVar(&args.partitions, "partitions", allPartitionsHuman, "comma separated list of partitions to limit offsets to, or all")
flags.BoolVar(&args.offsets, "offsets", true, "Controls if offsets should be fetched (defaults to true)")
flags.Usage = func() {
fmt.Fprintln(os.Stderr, "Usage of group:")
flags.PrintDefaults()
fmt.Fprintln(os.Stderr, groupDocString)
}
err := flags.Parse(as)
if err != nil && strings.Contains(err.Error(), "flag: help requested") {
os.Exit(0)
} else if err != nil {
os.Exit(2)
}
return args
}
var groupDocString = fmt.Sprintf(`
The values for -topic and -brokers can also be set via environment variables %s and %s respectively.
The values supplied on the command line win over environment variable values.
The group command can be used to list groups, their offsets and lag and to reset a group's offset.
When an explicit offset hasn't been set yet, kt prints out the respective sarama constants, cf. https://godoc.org/github.com/IBM/sarama#pkg-constants
To simply list all groups:
kt group
This is faster when not fetching offsets:
kt group -offsets=false
To filter by regex:
kt group -filter specials
To filter by topic:
kt group -topic fav-topic
To reset a consumer group's offset to a specific value on a single partition:
kt group -reset 23 -topic fav-topic -group specials -partitions 2
To reset a consumer group's offset to the newest currently available on all partitions:
kt group -reset newest -topic fav-topic -group specials -partitions all
To reset a consumer group's offset to the newest that was available at a specific time on all partitions:
kt group -reset "6 hours ago" -topic fav-topic -group specials -partitions all
See https://github.com/markusmobius/go-dateparser for a list of supported time formats.
`, ENV_TOPIC, ENV_BROKERS)