-
Notifications
You must be signed in to change notification settings - Fork 658
/
admin_elect_leaders.go
113 lines (98 loc) · 2.92 KB
/
admin_elect_leaders.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
/**
* Copyright 2024 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.
*/
// Perform Preferred or Unclean Elections for the specified Topic Partitions.
package main
import (
"context"
"fmt"
"os"
"strconv"
"time"
"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)
func main() {
args := os.Args
// Providing an empty topic partition list will give an empty list as result.
if len(args) < 2 {
fmt.Fprintf(os.Stderr,
"Usage: %s <bootstrap_servers> "+
"<election_type (Preferred/Unclean)> <topic1> <partition1> ...\n",
args[0])
os.Exit(1)
}
// Create new AdminClient.
ac, err := kafka.NewAdminClient(&kafka.ConfigMap{
"bootstrap.servers": args[1],
})
if err != nil {
fmt.Printf("Failed to create Admin client: %s\n", err)
os.Exit(1)
}
defer ac.Close()
electionType, err := kafka.ElectionTypeFromString(args[2])
if err != nil {
fmt.Printf("Invalid election type: %s\n", err)
os.Exit(1)
}
var topicPartitionList []kafka.TopicPartition
argsCnt := len(os.Args)
i := 3
index := 0
for i < argsCnt {
if i+2 > argsCnt {
fmt.Printf("Expected %d arguments for partition %d, got %d\n", 2, index, argsCnt-i)
os.Exit(1)
}
topicName := os.Args[i]
partition, err := strconv.ParseInt(args[i+1], 10, 32)
if err != nil {
panic(err)
}
topicPartitionList = append(topicPartitionList, kafka.TopicPartition{
Topic: &topicName,
Partition: int32(partition),
})
i += 2
index++
}
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
res, err := ac.ElectLeaders(ctx, kafka.NewElectLeadersRequest(electionType, topicPartitionList))
if err != nil {
kafkaErr, ok := err.(kafka.Error)
if ok && kafkaErr.Code() != kafka.ErrNoError {
fmt.Printf("Failed to elect Leaders: %s\n", err)
os.Exit(1)
}
}
fmt.Printf("ElectLeaders result contains %d partition(s):\n",
len(res.TopicPartitions))
for _, topicPartition := range res.TopicPartitions {
kafkaErr, ok := topicPartition.Error.(kafka.Error)
if !ok {
fmt.Printf("Topic: %s, Partition: %d - Unexpected error type: %s\n",
*topicPartition.Topic, topicPartition.Partition, err)
continue
}
if kafkaErr.Code() == kafka.ErrNoError {
fmt.Printf("Topic: %s, Partition: %d - Success\n",
*topicPartition.Topic, topicPartition.Partition)
} else {
fmt.Printf("Topic: %s, Partition: %d - Failed: %s\n",
*topicPartition.Topic, topicPartition.Partition, kafkaErr.String())
}
}
}