/
topic-operation.go
705 lines (571 loc) · 19.1 KB
/
topic-operation.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
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
package topic
import (
"fmt"
"sort"
"strconv"
"strings"
"time"
"github.com/Shopify/sarama"
"github.com/deviceinsight/kafkactl/internal"
"github.com/deviceinsight/kafkactl/output"
"github.com/deviceinsight/kafkactl/util"
"github.com/pkg/errors"
"github.com/spf13/cobra"
"gopkg.in/yaml.v2"
)
type Topic struct {
Name string
Partitions []Partition `json:",omitempty" yaml:",omitempty"`
Configs []internal.Config `json:",omitempty" yaml:",omitempty"`
}
type Partition struct {
ID int32
OldestOffset int64 `json:"oldestOffset" yaml:"oldestOffset"`
NewestOffset int64 `json:"newestOffset" yaml:"newestOffset"`
Leader string `json:",omitempty" yaml:",omitempty"`
Replicas []int32 `json:",omitempty" yaml:",omitempty,flow"`
ISRs []int32 `json:"inSyncReplicas,omitempty" yaml:"inSyncReplicas,omitempty,flow"`
}
type requestedTopicFields struct {
partitionID bool
partitionOffset bool
partitionLeader bool
partitionReplicas bool
partitionISRs bool
config bool
}
var allFields = requestedTopicFields{partitionID: true, partitionOffset: true, partitionLeader: true, partitionReplicas: true, partitionISRs: true, config: true}
type GetTopicsFlags struct {
OutputFormat string
}
type CreateTopicFlags struct {
Partitions int32
ReplicationFactor int16
ValidateOnly bool
Configs []string
}
type AlterTopicFlags struct {
Partitions int32
ReplicationFactor int16
ValidateOnly bool
Configs []string
}
type DescribeTopicFlags struct {
PrintConfigs bool
SkipEmptyPartitions bool
OutputFormat string
}
type Operation struct {
}
func (operation *Operation) CreateTopics(topics []string, flags CreateTopicFlags) error {
var (
err error
context internal.ClientContext
admin sarama.ClusterAdmin
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
topicDetails := sarama.TopicDetail{
NumPartitions: flags.Partitions,
ReplicationFactor: flags.ReplicationFactor,
ConfigEntries: map[string]*string{},
}
for _, config := range flags.Configs {
configParts := strings.Split(config, "=")
topicDetails.ConfigEntries[configParts[0]] = &configParts[1]
}
for _, topic := range topics {
if err = admin.CreateTopic(topic, &topicDetails, flags.ValidateOnly); err != nil {
return errors.Wrap(err, "failed to create topic")
}
output.Infof("topic created: %s", topic)
}
return nil
}
func (operation *Operation) DeleteTopics(topics []string) error {
var (
err error
context internal.ClientContext
admin sarama.ClusterAdmin
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
for _, topic := range topics {
if err = admin.DeleteTopic(topic); err != nil {
return errors.Wrap(err, "failed to delete topic")
}
output.Infof("topic deleted: %s", topic)
}
return nil
}
func (operation *Operation) DescribeTopic(topic string, flags DescribeTopicFlags) error {
var (
context internal.ClientContext
client sarama.Client
admin sarama.ClusterAdmin
err error
exists bool
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if client, err = internal.CreateClient(&context); err != nil {
return errors.Wrap(err, "failed to create client")
}
if exists, err = internal.TopicExists(&client, topic); err != nil {
return errors.Wrap(err, "failed to read topics")
}
if !exists {
return errors.Errorf("topic '%s' does not exist", topic)
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
var t, _ = readTopic(&client, &admin, topic, allFields)
return operation.printTopic(t, flags)
}
func (operation *Operation) printTopic(topic Topic, flags DescribeTopicFlags) error {
if flags.PrintConfigs {
if flags.OutputFormat == "json" || flags.OutputFormat == "yaml" {
topic.Configs = nil
} else {
configTableWriter := output.CreateTableWriter()
if err := configTableWriter.WriteHeader("CONFIG", "VALUE"); err != nil {
return err
}
for _, c := range topic.Configs {
if err := configTableWriter.Write(c.Name, c.Value); err != nil {
return err
}
}
if err := configTableWriter.Flush(); err != nil {
return err
}
output.PrintStrings("")
}
}
if flags.SkipEmptyPartitions {
partitionsWithMessages := make([]Partition, 0)
for _, p := range topic.Partitions {
if p.OldestOffset < p.NewestOffset {
partitionsWithMessages = append(partitionsWithMessages, p)
}
}
topic.Partitions = partitionsWithMessages
}
partitionTableWriter := output.CreateTableWriter()
if flags.OutputFormat == "" || flags.OutputFormat == "wide" {
if err := partitionTableWriter.WriteHeader("PARTITION", "OLDEST_OFFSET", "NEWEST_OFFSET", "EMPTY",
"LEADER", "REPLICAS", "IN_SYNC_REPLICAS"); err != nil {
return err
}
} else if flags.OutputFormat != "json" && flags.OutputFormat != "yaml" {
return errors.Errorf("unknown outputFormat: %s", flags.OutputFormat)
}
if flags.OutputFormat == "json" || flags.OutputFormat == "yaml" {
return output.PrintObject(topic, flags.OutputFormat)
} else if flags.OutputFormat == "wide" || flags.OutputFormat == "" {
for _, p := range topic.Partitions {
replicas := strings.Trim(strings.Join(strings.Fields(fmt.Sprint(p.Replicas)), ","), "[]")
inSyncReplicas := strings.Trim(strings.Join(strings.Fields(fmt.Sprint(p.ISRs)), ","), "[]")
if err := partitionTableWriter.Write(strconv.Itoa(int(p.ID)), strconv.Itoa(int(p.OldestOffset)),
strconv.Itoa(int(p.NewestOffset)), strconv.FormatBool((p.NewestOffset-p.OldestOffset) <= 0), p.Leader, replicas, inSyncReplicas); err != nil {
return err
}
}
}
if flags.OutputFormat == "" || flags.OutputFormat == "wide" {
if err := partitionTableWriter.Flush(); err != nil {
return err
}
}
return nil
}
func (operation *Operation) AlterTopic(topic string, flags AlterTopicFlags) error {
var (
context internal.ClientContext
client sarama.Client
admin sarama.ClusterAdmin
err error
exists bool
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if client, err = internal.CreateClient(&context); err != nil {
return errors.Wrap(err, "failed to create client")
}
if exists, err = internal.TopicExists(&client, topic); err != nil {
return errors.Wrap(err, "failed to read topics")
}
if !exists {
return errors.Errorf("topic '%s' does not exist", topic)
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
var t, _ = readTopic(&client, &admin, topic, allFields)
if flags.Partitions != 0 {
if len(t.Partitions) > int(flags.Partitions) {
return errors.New("Decreasing the number of partitions is not supported")
} else if len(t.Partitions) == int(flags.Partitions) {
return errors.Errorf("Topic already has %d partitions", flags.Partitions)
}
if flags.ValidateOnly {
for len(t.Partitions) < int(flags.Partitions) {
t.Partitions = append(t.Partitions, Partition{ID: int32(len(t.Partitions)), NewestOffset: 0, OldestOffset: 0})
}
} else {
var emptyAssignment = make([][]int32, 0)
err = admin.CreatePartitions(topic, flags.Partitions, emptyAssignment, flags.ValidateOnly)
if err != nil {
return errors.Errorf("Could not create partitions for topic '%s': %v", topic, err)
}
output.Infof("partitions have been created")
}
}
if flags.ReplicationFactor > 0 {
var brokers = client.Brokers()
if int(flags.ReplicationFactor) > len(brokers) {
return errors.Errorf("Replication factor for topic '%s' must not exceed the number of available brokers", topic)
}
brokerReplicaCount := make(map[int32]int)
for _, broker := range brokers {
brokerReplicaCount[broker.ID()] = 0
}
for _, partition := range t.Partitions {
for _, brokerID := range partition.Replicas {
brokerReplicaCount[brokerID]++
}
}
var replicaAssignment = make([][]int32, 0, int16(len(t.Partitions)))
for _, partition := range t.Partitions {
var replicas, err = getTargetReplicas(partition.Replicas, brokerReplicaCount, flags.ReplicationFactor)
if err != nil {
return errors.Wrap(err, "unable to determine target replicas")
}
replicaAssignment = append(replicaAssignment, replicas)
}
for brokerID, replicaCount := range brokerReplicaCount {
output.Debugf("broker %d now has %d replicas", brokerID, replicaCount)
}
if flags.ValidateOnly {
for i := range t.Partitions {
t.Partitions[i].Replicas = replicaAssignment[i]
}
} else {
err = admin.AlterPartitionReassignments(topic, replicaAssignment)
if err != nil {
return errors.Errorf("Could not reassign partition replicas for topic '%s': %v", topic, err)
}
partitions := make([]int32, len(t.Partitions))
for _, p := range t.Partitions {
partitions[0] = p.ID
}
assignmentRunning := true
for assignmentRunning {
status, err := admin.ListPartitionReassignments(topic, partitions)
if err != nil {
return errors.Errorf("Could query reassignment status for topic '%s': %v", topic, err)
}
assignmentRunning = false
if statusTopic, ok := status[topic]; ok {
for partitionID, statusPartition := range statusTopic {
output.Infof("reassignment running for topic=%s partition=%d: replicas:%v addingReplicas:%v removingReplicas:%v",
topic, partitionID, statusPartition.Replicas, statusPartition.AddingReplicas, statusPartition.RemovingReplicas)
time.Sleep(5 * time.Second)
assignmentRunning = true
}
} else {
output.Debugf("Emtpy list partition reassignment result returned (len status: %d)", len(status))
}
}
output.Infof("partition replicas have been reassigned")
}
}
if len(flags.Configs) > 0 {
mergedConfigEntries := make(map[string]*string)
for i, config := range t.Configs {
mergedConfigEntries[config.Name] = &(t.Configs[i].Value)
}
for _, config := range flags.Configs {
configParts := strings.Split(config, "=")
if len(configParts) == 2 {
if len(configParts[1]) == 0 {
delete(mergedConfigEntries, configParts[0])
} else {
mergedConfigEntries[configParts[0]] = &configParts[1]
}
}
}
if flags.ValidateOnly {
// validate only - directly alter the response object
t.Configs = make([]internal.Config, 0, len(mergedConfigEntries))
for key, value := range mergedConfigEntries {
t.Configs = append(t.Configs, internal.Config{Name: key, Value: *value})
}
} else {
if err = admin.AlterConfig(sarama.TopicResource, topic, mergedConfigEntries, flags.ValidateOnly); err != nil {
return errors.Errorf("Could not alter topic config '%s': %v", topic, err)
}
output.Infof("config has been altered")
}
}
if flags.ValidateOnly {
describeFlags := DescribeTopicFlags{PrintConfigs: len(flags.Configs) > 0}
return operation.printTopic(t, describeFlags)
}
return nil
}
func (operation *Operation) ListTopicsNames() ([]string, error) {
var (
err error
context internal.ClientContext
client sarama.Client
topics []string
)
if context, err = internal.CreateClientContext(); err != nil {
return nil, err
}
if client, err = internal.CreateClient(&context); err != nil {
return nil, errors.Wrap(err, "failed to create client")
}
if topics, err = client.Topics(); err != nil {
return nil, errors.Wrap(err, "failed to read topics")
}
return topics, nil
}
func getTargetReplicas(currentReplicas []int32, brokerReplicaCount map[int32]int, targetReplicationFactor int16) ([]int32, error) {
replicas := currentReplicas
for len(replicas) > int(targetReplicationFactor) {
sort.Slice(replicas, func(i, j int) bool {
brokerI := replicas[i]
brokerJ := replicas[j]
return brokerReplicaCount[brokerI] < brokerReplicaCount[brokerJ] || (brokerReplicaCount[brokerI] == brokerReplicaCount[brokerJ] && brokerI < brokerJ)
})
lastReplica := replicas[len(replicas)-1]
replicas = replicas[:len(replicas)-1]
brokerReplicaCount[lastReplica]--
}
var unusedBrokerIds []int32
if len(replicas) < int(targetReplicationFactor) {
for brokerID := range brokerReplicaCount {
if !util.ContainsInt32(replicas, brokerID) {
unusedBrokerIds = append(unusedBrokerIds, brokerID)
}
}
if len(unusedBrokerIds) < (int(targetReplicationFactor) - len(replicas)) {
return nil, errors.New("not enough brokers")
}
}
for len(replicas) < int(targetReplicationFactor) {
sort.Slice(unusedBrokerIds, func(i, j int) bool {
brokerI := unusedBrokerIds[i]
brokerJ := unusedBrokerIds[j]
return brokerReplicaCount[brokerI] < brokerReplicaCount[brokerJ] || (brokerReplicaCount[brokerI] == brokerReplicaCount[brokerJ] && brokerI > brokerJ)
})
replicas = append(replicas, unusedBrokerIds[0])
brokerReplicaCount[unusedBrokerIds[0]]++
unusedBrokerIds = unusedBrokerIds[1:]
}
return replicas, nil
}
func (operation *Operation) GetTopics(flags GetTopicsFlags) error {
var (
err error
context internal.ClientContext
client sarama.Client
admin sarama.ClusterAdmin
topics []string
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
if client, err = internal.CreateClient(&context); err != nil {
return errors.Wrap(err, "failed to create client")
}
if topics, err = client.Topics(); err != nil {
return errors.Wrap(err, "failed to read topics")
}
tableWriter := output.CreateTableWriter()
var requestedFields requestedTopicFields
if flags.OutputFormat == "" {
requestedFields = requestedTopicFields{partitionID: true}
if err := tableWriter.WriteHeader("TOPIC", "PARTITIONS"); err != nil {
return err
}
} else if flags.OutputFormat == "compact" {
tableWriter.Initialize()
} else if flags.OutputFormat == "wide" {
requestedFields = requestedTopicFields{partitionID: true, config: true}
if err := tableWriter.WriteHeader("TOPIC", "PARTITIONS", "CONFIGS"); err != nil {
return err
}
} else if flags.OutputFormat == "json" {
requestedFields = allFields
} else if flags.OutputFormat == "yaml" {
requestedFields = allFields
} else {
return errors.Errorf("unknown outputFormat: %s", flags.OutputFormat)
}
topicChannel := make(chan Topic)
errChannel := make(chan error)
// read topics in parallel
for _, topic := range topics {
go func(topic string) {
t, err := readTopic(&client, &admin, topic, requestedFields)
if err != nil {
errChannel <- errors.Errorf("unable to read topic %s: %v", topic, err)
} else {
topicChannel <- t
}
}(topic)
}
topicList := make([]Topic, 0, len(topics))
for range topics {
select {
case topic := <-topicChannel:
topicList = append(topicList, topic)
case err := <-errChannel:
return err
}
}
sort.Slice(topicList, func(i, j int) bool {
return topicList[i].Name < topicList[j].Name
})
if flags.OutputFormat == "json" || flags.OutputFormat == "yaml" {
return output.PrintObject(topicList, flags.OutputFormat)
} else if flags.OutputFormat == "wide" {
for _, t := range topicList {
if err := tableWriter.Write(t.Name, strconv.Itoa(len(t.Partitions)), getConfigString(t.Configs)); err != nil {
return err
}
}
} else if flags.OutputFormat == "compact" {
for _, t := range topicList {
if err := tableWriter.Write(t.Name); err != nil {
return err
}
}
} else {
for _, t := range topicList {
if err := tableWriter.Write(t.Name, strconv.Itoa(len(t.Partitions))); err != nil {
return err
}
}
}
if flags.OutputFormat == "wide" || flags.OutputFormat == "compact" || flags.OutputFormat == "" {
if err := tableWriter.Flush(); err != nil {
return err
}
}
return nil
}
func readTopic(client *sarama.Client, admin *sarama.ClusterAdmin, name string, requestedFields requestedTopicFields) (Topic, error) {
var (
err error
ps []int32
led *sarama.Broker
top = Topic{Name: name}
)
if !requestedFields.partitionID {
return top, nil
}
if ps, err = (*client).Partitions(name); err != nil {
return top, err
}
partitionChannel := make(chan Partition)
errChannel := make(chan error)
// read partitions in parallel
for _, p := range ps {
go func(partitionId int32) {
np := Partition{ID: partitionId}
if requestedFields.partitionOffset {
if np.OldestOffset, err = (*client).GetOffset(name, partitionId, sarama.OffsetOldest); err != nil {
errChannel <- errors.Errorf("unable to read oldest offset for topic %s partition %d", name, partitionId)
return
}
if np.NewestOffset, err = (*client).GetOffset(name, partitionId, sarama.OffsetNewest); err != nil {
errChannel <- errors.Errorf("unable to read newest offset for topic %s partition %d", name, partitionId)
return
}
}
if requestedFields.partitionLeader {
if led, err = (*client).Leader(name, partitionId); err != nil {
errChannel <- errors.Errorf("unable to read leader for topic %s partition %d", name, partitionId)
return
}
np.Leader = led.Addr()
}
if requestedFields.partitionReplicas {
if np.Replicas, err = (*client).Replicas(name, partitionId); err != nil {
errChannel <- errors.Errorf("unable to read replicas for topic %s partition %d", name, partitionId)
return
}
sort.Slice(np.Replicas, func(i, j int) bool { return np.Replicas[i] < np.Replicas[j] })
}
if requestedFields.partitionISRs {
if np.ISRs, err = (*client).InSyncReplicas(name, partitionId); err != nil {
errChannel <- errors.Errorf("unable to read inSyncReplicas for topic %s partition %d", name, partitionId)
return
}
sort.Slice(np.ISRs, func(i, j int) bool { return np.ISRs[i] < np.ISRs[j] })
}
partitionChannel <- np
}(p)
}
for range ps {
select {
case partition := <-partitionChannel:
top.Partitions = append(top.Partitions, partition)
case err := <-errChannel:
return top, err
}
}
sort.Slice(top.Partitions, func(i, j int) bool {
return top.Partitions[i].ID < top.Partitions[j].ID
})
if requestedFields.config {
topicConfig := sarama.ConfigResource{
Type: sarama.TopicResource,
Name: name,
}
if top.Configs, err = internal.ListConfigs(admin, topicConfig); err != nil {
return top, err
}
}
return top, nil
}
func getConfigString(configs []internal.Config) string {
var configStrings []string
for _, config := range configs {
configStrings = append(configStrings, fmt.Sprintf("%s=%s", config.Name, config.Value))
}
return strings.Trim(strings.Join(configStrings, ","), "[]")
}
func CompleteTopicNames(_ *cobra.Command, args []string, _ string) ([]string, cobra.ShellCompDirective) {
if len(args) != 0 {
return nil, cobra.ShellCompDirectiveNoFileComp
}
topics, err := (&Operation{}).ListTopicsNames()
if err != nil {
return nil, cobra.ShellCompDirectiveError
}
return topics, cobra.ShellCompDirectiveNoFileComp
}
func FromYaml(yamlString string) (Topic, error) {
var t Topic
err := yaml.Unmarshal([]byte(yamlString), &t)
return t, err
}