forked from bsm/sarama-cluster
-
Notifications
You must be signed in to change notification settings - Fork 0
/
balancer.go
144 lines (122 loc) · 3.39 KB
/
balancer.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
package cluster
import (
"math"
"sort"
"gopkg.in/Shopify/sarama.v1"
)
// Notification events are emitted by the consumers on rebalancing
type Notification struct {
// Claimed contains topic/partitions that were claimed by this rebalance cycle
Claimed map[string][]int32
// Released contains topic/partitions that were released as part of this rebalance cycle
Released map[string][]int32
// Current are topic/partitions that are currently claimed to the consumer
Current map[string][]int32
}
func newNotification(released map[string][]int32) *Notification {
return &Notification{
Claimed: make(map[string][]int32),
Released: released,
Current: make(map[string][]int32),
}
}
func (n *Notification) claim(current map[string][]int32) {
previous := n.Released
for topic, partitions := range current {
n.Claimed[topic] = int32Slice(partitions).Diff(int32Slice(previous[topic]))
}
for topic, partitions := range previous {
n.Released[topic] = int32Slice(partitions).Diff(int32Slice(current[topic]))
}
n.Current = current
}
// --------------------------------------------------------------------
type topicInfo struct {
Partitions []int32
MemberIDs []string
}
func (info topicInfo) Perform(s Strategy) map[string][]int32 {
if s == StrategyRoundRobin {
return info.RoundRobin()
}
return info.Ranges()
}
func (info topicInfo) Ranges() map[string][]int32 {
sort.Strings(info.MemberIDs)
mlen := len(info.MemberIDs)
plen := len(info.Partitions)
res := make(map[string][]int32, mlen)
for pos, memberID := range info.MemberIDs {
n, i := float64(plen)/float64(mlen), float64(pos)
min := int(math.Floor(i*n + 0.5))
max := int(math.Floor((i+1)*n + 0.5))
sub := info.Partitions[min:max]
if len(sub) > 0 {
res[memberID] = sub
}
}
return res
}
func (info topicInfo) RoundRobin() map[string][]int32 {
sort.Strings(info.MemberIDs)
mlen := len(info.MemberIDs)
res := make(map[string][]int32, mlen)
for i, pnum := range info.Partitions {
memberID := info.MemberIDs[i%mlen]
res[memberID] = append(res[memberID], pnum)
}
return res
}
// --------------------------------------------------------------------
type balancer struct {
client sarama.Client
topics map[string]topicInfo
}
func newBalancerFromMeta(client sarama.Client, members map[string]sarama.ConsumerGroupMemberMetadata) (*balancer, error) {
balancer := newBalancer(client)
for memberID, meta := range members {
for _, topic := range meta.Topics {
if err := balancer.Topic(topic, memberID); err != nil {
return nil, err
}
}
}
return balancer, nil
}
func newBalancer(client sarama.Client) *balancer {
return &balancer{
client: client,
topics: make(map[string]topicInfo),
}
}
func (r *balancer) Topic(name string, memberID string) error {
topic, ok := r.topics[name]
if !ok {
nums, err := r.client.Partitions(name)
if err != nil {
return err
}
topic = topicInfo{
Partitions: nums,
MemberIDs: make([]string, 0, 1),
}
}
topic.MemberIDs = append(topic.MemberIDs, memberID)
r.topics[name] = topic
return nil
}
func (r *balancer) Perform(s Strategy) map[string]map[string][]int32 {
if r == nil {
return nil
}
res := make(map[string]map[string][]int32, 1)
for topic, info := range r.topics {
for memberID, partitions := range info.Perform(s) {
if _, ok := res[memberID]; !ok {
res[memberID] = make(map[string][]int32, 1)
}
res[memberID][topic] = partitions
}
}
return res
}