-
Notifications
You must be signed in to change notification settings - Fork 73
/
consumer-group-offset-operation.go
323 lines (263 loc) · 8.61 KB
/
consumer-group-offset-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
package consumergroupoffsets
import (
"github.com/deviceinsight/kafkactl/v5/internal/helpers"
"golang.org/x/sync/errgroup"
"github.com/IBM/sarama"
"github.com/deviceinsight/kafkactl/v5/internal"
"github.com/deviceinsight/kafkactl/v5/internal/output"
"github.com/deviceinsight/kafkactl/v5/internal/util"
"github.com/pkg/errors"
)
type ResetConsumerGroupOffsetFlags struct {
Topic []string
AllTopics bool
Partition int32
Offset int64
OldestOffset bool
NewestOffset bool
Execute bool
OutputFormat string
allowedGroupState string
ToDatetime string
}
type ConsumerGroupOffsetOperation struct {
}
func (operation *ConsumerGroupOffsetOperation) ResetConsumerGroupOffset(flags ResetConsumerGroupOffsetFlags, groupName string) error {
if (flags.Topic == nil || len(flags.Topic) == 0) && (!flags.AllTopics) {
return errors.New("no topic specified")
}
if !flags.Execute {
output.Warnf("nothing will be changed (include --execute to perform the reset)")
}
var (
ctx internal.ClientContext
config *sarama.Config
err error
client sarama.Client
admin sarama.ClusterAdmin
descriptions []*sarama.GroupDescription
)
if ctx, err = internal.CreateClientContext(); err != nil {
return err
}
if config, err = internal.CreateClientConfig(&ctx); err != nil {
return err
}
if client, err = internal.CreateClient(&ctx); err != nil {
return errors.Wrap(err, "failed to create client")
}
if admin, err = internal.CreateClusterAdmin(&ctx); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
var topics []string
if flags.AllTopics {
// retrieve all topics in the consumerGroup
offsets, err := admin.ListConsumerGroupOffsets(groupName, nil)
if err != nil {
return errors.Wrap(err, "failed to list consumer group offsets")
}
for topic := range offsets.Blocks {
topics = append(topics, topic)
}
} else {
// verify that the provided topics exist
existingTopics, err := client.Topics()
if err != nil {
return errors.Wrap(err, "failed to list available topics")
}
for _, topic := range flags.Topic {
if !util.ContainsString(existingTopics, topic) {
return errors.Errorf("topic does not exist: %s", topic)
}
}
topics = flags.Topic
}
output.Debugf("reset consumer-group offset for topics: %v", topics)
if flags.allowedGroupState == "" {
// a reset is only allowed if group is empty (no one in the group)
// for creation of the group state "Dead" is allowed
flags.allowedGroupState = "Empty"
}
if flags.Execute {
if descriptions, err = admin.DescribeConsumerGroups([]string{groupName}); err != nil {
return errors.Wrap(err, "failed to describe consumer group")
}
for _, description := range descriptions {
// https://stackoverflow.com/a/61745884/1115279
if description.State != flags.allowedGroupState {
return errors.Errorf("cannot reset offsets on consumer group %s. There are consumers assigned (state: %s)", groupName, description.State)
}
}
}
consumerGroup, err := sarama.NewConsumerGroup(ctx.Brokers, groupName, config)
if err != nil {
return errors.Errorf("failed to create consumer group %s: %v", groupName, err)
}
terminalCtx := helpers.CreateTerminalContext()
consumeErrorGroup, _ := errgroup.WithContext(terminalCtx)
consumeErrorGroup.SetLimit(100)
for _, topic := range topics {
topicName := topic
consumeErrorGroup.Go(func() error {
consumer := OffsetResettingConsumer{
client: client,
groupName: groupName,
topicName: topicName,
flags: flags,
ready: make(chan bool),
}
err = consumerGroup.Consume(terminalCtx, []string{topicName}, &consumer)
if err != nil {
return err
}
<-consumer.ready
return nil
})
}
err = consumeErrorGroup.Wait()
if err != nil {
return err
}
err = consumerGroup.Close()
if err != nil {
return err
}
return nil
}
func (operation *ConsumerGroupOffsetOperation) CreateConsumerGroup(flags ResetConsumerGroupOffsetFlags, group string) error {
flags.allowedGroupState = "Dead"
flags.Execute = true
flags.OutputFormat = "none"
err := operation.ResetConsumerGroupOffset(flags, group)
if err != nil {
return err
}
output.Infof("consumer-group created: %s", group)
return nil
}
type PartitionOffset struct {
Offset int64
Metadata string
}
func (operation *ConsumerGroupOffsetOperation) CloneConsumerGroup(srcGroup, targetGroup string) error {
var (
err error
context internal.ClientContext
config *sarama.Config
admin sarama.ClusterAdmin
srcOffsets, targetOffsets *sarama.OffsetFetchResponse
)
if context, err = internal.CreateClientContext(); err != nil {
return err
}
if config, err = internal.CreateClientConfig(&context); err != nil {
return err
}
if admin, err = internal.CreateClusterAdmin(&context); err != nil {
return errors.Wrap(err, "failed to create cluster admin")
}
if srcOffsets, err = admin.ListConsumerGroupOffsets(srcGroup, nil); err != nil {
return errors.Wrapf(err, "failed to get consumerGroup '%s' offsets", srcGroup)
}
if len(srcOffsets.Blocks) == 0 {
return errors.Errorf("consumerGroup '%s' does not contain offsets", srcGroup)
}
if targetOffsets, err = admin.ListConsumerGroupOffsets(targetGroup, nil); err != nil {
return errors.Wrapf(err, "failed to get consumerGroup '%s' offsets", targetGroup)
}
if len(targetOffsets.Blocks) != 0 {
return errors.Errorf("consumerGroup '%s' contains offsets", targetGroup)
}
topicPartitionOffsets := make(map[string]map[int32]PartitionOffset) // topic->partition->offset
for topic, partitions := range srcOffsets.Blocks {
p := topicPartitionOffsets[topic]
if p == nil {
p = make(map[int32]PartitionOffset)
}
for partition, block := range partitions {
p[partition] = PartitionOffset{Offset: block.Offset, Metadata: block.Metadata}
}
topicPartitionOffsets[topic] = p
}
consumerGroup, err := sarama.NewConsumerGroup(context.Brokers, targetGroup, config)
if err != nil {
return errors.Errorf("failed to create consumer group %s: %v", targetGroup, err)
}
terminalCtx := helpers.CreateTerminalContext()
consumeErrorGroup, _ := errgroup.WithContext(terminalCtx)
consumeErrorGroup.SetLimit(100)
for topic, partitionOffsets := range topicPartitionOffsets {
topicName, offsets := topic, partitionOffsets
consumeErrorGroup.Go(func() error {
consumer := OffsetSettingConsumer{
Topic: topicName,
PartitionOffsets: offsets,
}
err = consumerGroup.Consume(terminalCtx, []string{topicName}, &consumer)
if err != nil {
return err
}
<-consumer.ready
return nil
})
}
err = consumeErrorGroup.Wait()
if err != nil {
return err
}
err = consumerGroup.Close()
if err != nil {
return err
}
output.Infof("consumer-group %s cloned to %s", srcGroup, targetGroup)
return nil
}
type DeleteConsumerGroupOffsetFlags struct {
Topic string
Partition int32
}
func (operation *ConsumerGroupOffsetOperation) DeleteConsumerGroupOffset(groupName string, flags DeleteConsumerGroupOffsetFlags) error {
var (
err error
context internal.ClientContext
admin sarama.ClusterAdmin
partitions []int32
)
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")
}
defer admin.Close()
offsets, err := admin.ListConsumerGroupOffsets(groupName, nil)
if err != nil {
return errors.Wrapf(err, "failed to list group offsets: %s", groupName)
}
if _, ok := offsets.Blocks[flags.Topic]; !ok {
return errors.Errorf("no offsets for topic: %s", flags.Topic)
}
if flags.Partition == -1 {
// delete all existing offsets
partitions = make([]int32, 0)
for k, block := range offsets.Blocks[flags.Topic] {
output.Infof("block : %s", block)
partitions = append(partitions, int32(k))
}
} else {
// check if the partition exists and delete only the offset for this partition
if _, ok := offsets.Blocks[flags.Topic][flags.Partition]; !ok {
return errors.Errorf("No offset for partition: %d", flags.Partition)
}
partitions = []int32{flags.Partition}
}
for _, partition := range partitions {
if err = admin.DeleteConsumerGroupOffset(groupName, flags.Topic, partition); err != nil {
return errors.Wrapf(err, "failed to delete consumer-group-offset [group: %s, topic: %s, partition: %d]",
groupName, flags.Topic, flags.Partition)
}
output.Infof("consumer-group-offset deleted: [group: %s, topic: %s, partition: %d]",
groupName, flags.Topic, partition)
}
return nil
}