-
Notifications
You must be signed in to change notification settings - Fork 792
/
balancer.go
353 lines (305 loc) · 10.9 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
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
package kafka
import (
"hash"
"hash/crc32"
"hash/fnv"
"math/rand"
"sort"
"sync"
)
// The Balancer interface provides an abstraction of the message distribution
// logic used by Writer instances to route messages to the partitions available
// on a kafka cluster.
//
// Balancers must be safe to use concurrently from multiple goroutines.
type Balancer interface {
// Balance receives a message and a set of available partitions and
// returns the partition number that the message should be routed to.
//
// An application should refrain from using a balancer to manage multiple
// sets of partitions (from different topics for examples), use one balancer
// instance for each partition set, so the balancer can detect when the
// partitions change and assume that the kafka topic has been rebalanced.
Balance(msg Message, partitions ...int) (partition int)
}
// BalancerFunc is an implementation of the Balancer interface that makes it
// possible to use regular functions to distribute messages across partitions.
type BalancerFunc func(Message, ...int) int
// Balance calls f, satisfies the Balancer interface.
func (f BalancerFunc) Balance(msg Message, partitions ...int) int {
return f(msg, partitions...)
}
// RoundRobin is an Balancer implementation that equally distributes messages
// across all available partitions. It can take an optional chunk size to send
// ChunkSize messages to the same partition before moving to the next partition.
// This can be used to improve batch sizes.
type RoundRobin struct {
ChunkSize int
// Use a 32 bits integer so RoundRobin values don't need to be aligned to
// apply increments.
counter uint32
mutex sync.Mutex
}
// Balance satisfies the Balancer interface.
func (rr *RoundRobin) Balance(msg Message, partitions ...int) int {
return rr.balance(partitions)
}
func (rr *RoundRobin) balance(partitions []int) int {
rr.mutex.Lock()
defer rr.mutex.Unlock()
if rr.ChunkSize < 1 {
rr.ChunkSize = 1
}
length := len(partitions)
counterNow := rr.counter
offset := int(counterNow / uint32(rr.ChunkSize))
rr.counter++
return partitions[offset%length]
}
// LeastBytes is a Balancer implementation that routes messages to the partition
// that has received the least amount of data.
//
// Note that no coordination is done between multiple producers, having good
// balancing relies on the fact that each producer using a LeastBytes balancer
// should produce well balanced messages.
type LeastBytes struct {
mutex sync.Mutex
counters []leastBytesCounter
}
type leastBytesCounter struct {
partition int
bytes uint64
}
// Balance satisfies the Balancer interface.
func (lb *LeastBytes) Balance(msg Message, partitions ...int) int {
lb.mutex.Lock()
defer lb.mutex.Unlock()
// partitions change
if len(partitions) != len(lb.counters) {
lb.counters = lb.makeCounters(partitions...)
}
minBytes := lb.counters[0].bytes
minIndex := 0
for i, c := range lb.counters[1:] {
if c.bytes < minBytes {
minIndex = i + 1
minBytes = c.bytes
}
}
c := &lb.counters[minIndex]
c.bytes += uint64(len(msg.Key)) + uint64(len(msg.Value))
return c.partition
}
func (lb *LeastBytes) makeCounters(partitions ...int) (counters []leastBytesCounter) {
counters = make([]leastBytesCounter, len(partitions))
for i, p := range partitions {
counters[i].partition = p
}
sort.Slice(counters, func(i int, j int) bool {
return counters[i].partition < counters[j].partition
})
return
}
var (
fnv1aPool = &sync.Pool{
New: func() interface{} {
return fnv.New32a()
},
}
)
// Hash is a Balancer that uses the provided hash function to determine which
// partition to route messages to. This ensures that messages with the same key
// are routed to the same partition.
//
// The logic to calculate the partition is:
//
// hasher.Sum32() % len(partitions) => partition
//
// By default, Hash uses the FNV-1a algorithm. This is the same algorithm used
// by the Sarama Producer and ensures that messages produced by kafka-go will
// be delivered to the same topics that the Sarama producer would be delivered to.
type Hash struct {
rr RoundRobin
Hasher hash.Hash32
// lock protects Hasher while calculating the hash code. It is assumed that
// the Hasher field is read-only once the Balancer is created, so as a
// performance optimization, reads of the field are not protected.
lock sync.Mutex
}
func (h *Hash) Balance(msg Message, partitions ...int) int {
if msg.Key == nil {
return h.rr.Balance(msg, partitions...)
}
hasher := h.Hasher
if hasher != nil {
h.lock.Lock()
defer h.lock.Unlock()
} else {
hasher = fnv1aPool.Get().(hash.Hash32)
defer fnv1aPool.Put(hasher)
}
hasher.Reset()
if _, err := hasher.Write(msg.Key); err != nil {
panic(err)
}
// uses same algorithm that Sarama's hashPartitioner uses
// note the type conversions here. if the uint32 hash code is not cast to
// an int32, we do not get the same result as sarama.
partition := int32(hasher.Sum32()) % int32(len(partitions))
if partition < 0 {
partition = -partition
}
return int(partition)
}
// ReferenceHash is a Balancer that uses the provided hash function to determine which
// partition to route messages to. This ensures that messages with the same key
// are routed to the same partition.
//
// The logic to calculate the partition is:
//
// (int32(hasher.Sum32()) & 0x7fffffff) % len(partitions) => partition
//
// By default, ReferenceHash uses the FNV-1a algorithm. This is the same algorithm as
// the Sarama NewReferenceHashPartitioner and ensures that messages produced by kafka-go will
// be delivered to the same topics that the Sarama producer would be delivered to.
type ReferenceHash struct {
rr randomBalancer
Hasher hash.Hash32
// lock protects Hasher while calculating the hash code. It is assumed that
// the Hasher field is read-only once the Balancer is created, so as a
// performance optimization, reads of the field are not protected.
lock sync.Mutex
}
func (h *ReferenceHash) Balance(msg Message, partitions ...int) int {
if msg.Key == nil {
return h.rr.Balance(msg, partitions...)
}
hasher := h.Hasher
if hasher != nil {
h.lock.Lock()
defer h.lock.Unlock()
} else {
hasher = fnv1aPool.Get().(hash.Hash32)
defer fnv1aPool.Put(hasher)
}
hasher.Reset()
if _, err := hasher.Write(msg.Key); err != nil {
panic(err)
}
// uses the same algorithm as the Sarama's referenceHashPartitioner.
// note the type conversions here. if the uint32 hash code is not cast to
// an int32, we do not get the same result as sarama.
partition := (int32(hasher.Sum32()) & 0x7fffffff) % int32(len(partitions))
return int(partition)
}
type randomBalancer struct {
mock int // mocked return value, used for testing
}
func (b randomBalancer) Balance(msg Message, partitions ...int) (partition int) {
if b.mock != 0 {
return b.mock
}
return partitions[rand.Int()%len(partitions)]
}
// CRC32Balancer is a Balancer that uses the CRC32 hash function to determine
// which partition to route messages to. This ensures that messages with the
// same key are routed to the same partition. This balancer is compatible with
// the built-in hash partitioners in librdkafka and the language bindings that
// are built on top of it, including the
// github.com/confluentinc/confluent-kafka-go Go package.
//
// With the Consistent field false (default), this partitioner is equivalent to
// the "consistent_random" setting in librdkafka. When Consistent is true, this
// partitioner is equivalent to the "consistent" setting. The latter will hash
// empty or nil keys into the same partition.
//
// Unless you are absolutely certain that all your messages will have keys, it's
// best to leave the Consistent flag off. Otherwise, you run the risk of
// creating a very hot partition.
type CRC32Balancer struct {
Consistent bool
random randomBalancer
}
func (b CRC32Balancer) Balance(msg Message, partitions ...int) (partition int) {
// NOTE: the crc32 balancers in librdkafka don't differentiate between nil
// and empty keys. both cases are treated as unset.
if len(msg.Key) == 0 && !b.Consistent {
return b.random.Balance(msg, partitions...)
}
idx := crc32.ChecksumIEEE(msg.Key) % uint32(len(partitions))
return partitions[idx]
}
// Murmur2Balancer is a Balancer that uses the Murmur2 hash function to
// determine which partition to route messages to. This ensures that messages
// with the same key are routed to the same partition. This balancer is
// compatible with the partitioner used by the Java library and by librdkafka's
// "murmur2" and "murmur2_random" partitioners.
//
// With the Consistent field false (default), this partitioner is equivalent to
// the "murmur2_random" setting in librdkafka. When Consistent is true, this
// partitioner is equivalent to the "murmur2" setting. The latter will hash
// nil keys into the same partition. Empty, non-nil keys are always hashed to
// the same partition regardless of configuration.
//
// Unless you are absolutely certain that all your messages will have keys, it's
// best to leave the Consistent flag off. Otherwise, you run the risk of
// creating a very hot partition.
//
// Note that the librdkafka documentation states that the "murmur2_random" is
// functionally equivalent to the default Java partitioner. That's because the
// Java partitioner will use a round robin balancer instead of random on nil
// keys. We choose librdkafka's implementation because it arguably has a larger
// install base.
type Murmur2Balancer struct {
Consistent bool
random randomBalancer
}
func (b Murmur2Balancer) Balance(msg Message, partitions ...int) (partition int) {
// NOTE: the murmur2 balancers in java and librdkafka treat a nil key as
// non-existent while treating an empty slice as a defined value.
if msg.Key == nil && !b.Consistent {
return b.random.Balance(msg, partitions...)
}
idx := (murmur2(msg.Key) & 0x7fffffff) % uint32(len(partitions))
return partitions[idx]
}
// Go port of the Java library's murmur2 function.
// https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L353
func murmur2(data []byte) uint32 {
length := len(data)
const (
seed uint32 = 0x9747b28c
// 'm' and 'r' are mixing constants generated offline.
// They're not really 'magic', they just happen to work well.
m = 0x5bd1e995
r = 24
)
// Initialize the hash to a random value
h := seed ^ uint32(length)
length4 := length / 4
for i := 0; i < length4; i++ {
i4 := i * 4
k := (uint32(data[i4+0]) & 0xff) + ((uint32(data[i4+1]) & 0xff) << 8) + ((uint32(data[i4+2]) & 0xff) << 16) + ((uint32(data[i4+3]) & 0xff) << 24)
k *= m
k ^= k >> r
k *= m
h *= m
h ^= k
}
// Handle the last few bytes of the input array
extra := length % 4
if extra >= 3 {
h ^= (uint32(data[(length & ^3)+2]) & 0xff) << 16
}
if extra >= 2 {
h ^= (uint32(data[(length & ^3)+1]) & 0xff) << 8
}
if extra >= 1 {
h ^= uint32(data[length & ^3]) & 0xff
h *= m
}
h ^= h >> 13
h *= m
h ^= h >> 15
return h
}