github.com/QuangHoangHao/kafka-go@v0.4.36/balancer.go (about) 1 package kafka 2 3 import ( 4 "hash" 5 "hash/crc32" 6 "hash/fnv" 7 "math/rand" 8 "sort" 9 "sync" 10 "sync/atomic" 11 ) 12 13 // The Balancer interface provides an abstraction of the message distribution 14 // logic used by Writer instances to route messages to the partitions available 15 // on a kafka cluster. 16 // 17 // Balancers must be safe to use concurrently from multiple goroutines. 18 type Balancer interface { 19 // Balance receives a message and a set of available partitions and 20 // returns the partition number that the message should be routed to. 21 // 22 // An application should refrain from using a balancer to manage multiple 23 // sets of partitions (from different topics for examples), use one balancer 24 // instance for each partition set, so the balancer can detect when the 25 // partitions change and assume that the kafka topic has been rebalanced. 26 Balance(msg Message, partitions ...int) (partition int) 27 } 28 29 // BalancerFunc is an implementation of the Balancer interface that makes it 30 // possible to use regular functions to distribute messages across partitions. 31 type BalancerFunc func(Message, ...int) int 32 33 // Balance calls f, satisfies the Balancer interface. 34 func (f BalancerFunc) Balance(msg Message, partitions ...int) int { 35 return f(msg, partitions...) 36 } 37 38 // RoundRobin is an Balancer implementation that equally distributes messages 39 // across all available partitions. 40 type RoundRobin struct { 41 // Use a 32 bits integer so RoundRobin values don't need to be aligned to 42 // apply atomic increments. 43 offset uint32 44 } 45 46 // Balance satisfies the Balancer interface. 47 func (rr *RoundRobin) Balance(msg Message, partitions ...int) int { 48 return rr.balance(partitions) 49 } 50 51 func (rr *RoundRobin) balance(partitions []int) int { 52 length := uint32(len(partitions)) 53 offset := atomic.AddUint32(&rr.offset, 1) - 1 54 return partitions[offset%length] 55 } 56 57 // LeastBytes is a Balancer implementation that routes messages to the partition 58 // that has received the least amount of data. 59 // 60 // Note that no coordination is done between multiple producers, having good 61 // balancing relies on the fact that each producer using a LeastBytes balancer 62 // should produce well balanced messages. 63 type LeastBytes struct { 64 mutex sync.Mutex 65 counters []leastBytesCounter 66 } 67 68 type leastBytesCounter struct { 69 partition int 70 bytes uint64 71 } 72 73 // Balance satisfies the Balancer interface. 74 func (lb *LeastBytes) Balance(msg Message, partitions ...int) int { 75 lb.mutex.Lock() 76 defer lb.mutex.Unlock() 77 78 // partitions change 79 if len(partitions) != len(lb.counters) { 80 lb.counters = lb.makeCounters(partitions...) 81 } 82 83 minBytes := lb.counters[0].bytes 84 minIndex := 0 85 86 for i, c := range lb.counters[1:] { 87 if c.bytes < minBytes { 88 minIndex = i + 1 89 minBytes = c.bytes 90 } 91 } 92 93 c := &lb.counters[minIndex] 94 c.bytes += uint64(len(msg.Key)) + uint64(len(msg.Value)) 95 return c.partition 96 } 97 98 func (lb *LeastBytes) makeCounters(partitions ...int) (counters []leastBytesCounter) { 99 counters = make([]leastBytesCounter, len(partitions)) 100 101 for i, p := range partitions { 102 counters[i].partition = p 103 } 104 105 sort.Slice(counters, func(i int, j int) bool { 106 return counters[i].partition < counters[j].partition 107 }) 108 return 109 } 110 111 var ( 112 fnv1aPool = &sync.Pool{ 113 New: func() interface{} { 114 return fnv.New32a() 115 }, 116 } 117 ) 118 119 // Hash is a Balancer that uses the provided hash function to determine which 120 // partition to route messages to. This ensures that messages with the same key 121 // are routed to the same partition. 122 // 123 // The logic to calculate the partition is: 124 // 125 // hasher.Sum32() % len(partitions) => partition 126 // 127 // By default, Hash uses the FNV-1a algorithm. This is the same algorithm used 128 // by the Sarama Producer and ensures that messages produced by kafka-go will 129 // be delivered to the same topics that the Sarama producer would be delivered to. 130 type Hash struct { 131 rr RoundRobin 132 Hasher hash.Hash32 133 134 // lock protects Hasher while calculating the hash code. It is assumed that 135 // the Hasher field is read-only once the Balancer is created, so as a 136 // performance optimization, reads of the field are not protected. 137 lock sync.Mutex 138 } 139 140 func (h *Hash) Balance(msg Message, partitions ...int) int { 141 if msg.Key == nil { 142 return h.rr.Balance(msg, partitions...) 143 } 144 145 hasher := h.Hasher 146 if hasher != nil { 147 h.lock.Lock() 148 defer h.lock.Unlock() 149 } else { 150 hasher = fnv1aPool.Get().(hash.Hash32) 151 defer fnv1aPool.Put(hasher) 152 } 153 154 hasher.Reset() 155 if _, err := hasher.Write(msg.Key); err != nil { 156 panic(err) 157 } 158 159 // uses same algorithm that Sarama's hashPartitioner uses 160 // note the type conversions here. if the uint32 hash code is not cast to 161 // an int32, we do not get the same result as sarama. 162 partition := int32(hasher.Sum32()) % int32(len(partitions)) 163 if partition < 0 { 164 partition = -partition 165 } 166 167 return int(partition) 168 } 169 170 // ReferenceHash is a Balancer that uses the provided hash function to determine which 171 // partition to route messages to. This ensures that messages with the same key 172 // are routed to the same partition. 173 // 174 // The logic to calculate the partition is: 175 // 176 // (int32(hasher.Sum32()) & 0x7fffffff) % len(partitions) => partition 177 // 178 // By default, ReferenceHash uses the FNV-1a algorithm. This is the same algorithm as 179 // the Sarama NewReferenceHashPartitioner and ensures that messages produced by kafka-go will 180 // be delivered to the same topics that the Sarama producer would be delivered to. 181 type ReferenceHash struct { 182 rr randomBalancer 183 Hasher hash.Hash32 184 185 // lock protects Hasher while calculating the hash code. It is assumed that 186 // the Hasher field is read-only once the Balancer is created, so as a 187 // performance optimization, reads of the field are not protected. 188 lock sync.Mutex 189 } 190 191 func (h *ReferenceHash) Balance(msg Message, partitions ...int) int { 192 if msg.Key == nil { 193 return h.rr.Balance(msg, partitions...) 194 } 195 196 hasher := h.Hasher 197 if hasher != nil { 198 h.lock.Lock() 199 defer h.lock.Unlock() 200 } else { 201 hasher = fnv1aPool.Get().(hash.Hash32) 202 defer fnv1aPool.Put(hasher) 203 } 204 205 hasher.Reset() 206 if _, err := hasher.Write(msg.Key); err != nil { 207 panic(err) 208 } 209 210 // uses the same algorithm as the Sarama's referenceHashPartitioner. 211 // note the type conversions here. if the uint32 hash code is not cast to 212 // an int32, we do not get the same result as sarama. 213 partition := (int32(hasher.Sum32()) & 0x7fffffff) % int32(len(partitions)) 214 return int(partition) 215 } 216 217 type randomBalancer struct { 218 mock int // mocked return value, used for testing 219 } 220 221 func (b randomBalancer) Balance(msg Message, partitions ...int) (partition int) { 222 if b.mock != 0 { 223 return b.mock 224 } 225 return partitions[rand.Int()%len(partitions)] 226 } 227 228 // CRC32Balancer is a Balancer that uses the CRC32 hash function to determine 229 // which partition to route messages to. This ensures that messages with the 230 // same key are routed to the same partition. This balancer is compatible with 231 // the built-in hash partitioners in librdkafka and the language bindings that 232 // are built on top of it, including the 233 // github.com/confluentinc/confluent-kafka-go Go package. 234 // 235 // With the Consistent field false (default), this partitioner is equivalent to 236 // the "consistent_random" setting in librdkafka. When Consistent is true, this 237 // partitioner is equivalent to the "consistent" setting. The latter will hash 238 // empty or nil keys into the same partition. 239 // 240 // Unless you are absolutely certain that all your messages will have keys, it's 241 // best to leave the Consistent flag off. Otherwise, you run the risk of 242 // creating a very hot partition. 243 type CRC32Balancer struct { 244 Consistent bool 245 random randomBalancer 246 } 247 248 func (b CRC32Balancer) Balance(msg Message, partitions ...int) (partition int) { 249 // NOTE: the crc32 balancers in librdkafka don't differentiate between nil 250 // and empty keys. both cases are treated as unset. 251 if len(msg.Key) == 0 && !b.Consistent { 252 return b.random.Balance(msg, partitions...) 253 } 254 255 idx := crc32.ChecksumIEEE(msg.Key) % uint32(len(partitions)) 256 return partitions[idx] 257 } 258 259 // Murmur2Balancer is a Balancer that uses the Murmur2 hash function to 260 // determine which partition to route messages to. This ensures that messages 261 // with the same key are routed to the same partition. This balancer is 262 // compatible with the partitioner used by the Java library and by librdkafka's 263 // "murmur2" and "murmur2_random" partitioners. / 264 // 265 // With the Consistent field false (default), this partitioner is equivalent to 266 // the "murmur2_random" setting in librdkafka. When Consistent is true, this 267 // partitioner is equivalent to the "murmur2" setting. The latter will hash 268 // nil keys into the same partition. Empty, non-nil keys are always hashed to 269 // the same partition regardless of configuration. 270 // 271 // Unless you are absolutely certain that all your messages will have keys, it's 272 // best to leave the Consistent flag off. Otherwise, you run the risk of 273 // creating a very hot partition. 274 // 275 // Note that the librdkafka documentation states that the "murmur2_random" is 276 // functionally equivalent to the default Java partitioner. That's because the 277 // Java partitioner will use a round robin balancer instead of random on nil 278 // keys. We choose librdkafka's implementation because it arguably has a larger 279 // install base. 280 type Murmur2Balancer struct { 281 Consistent bool 282 random randomBalancer 283 } 284 285 func (b Murmur2Balancer) Balance(msg Message, partitions ...int) (partition int) { 286 // NOTE: the murmur2 balancers in java and librdkafka treat a nil key as 287 // non-existent while treating an empty slice as a defined value. 288 if msg.Key == nil && !b.Consistent { 289 return b.random.Balance(msg, partitions...) 290 } 291 292 idx := (murmur2(msg.Key) & 0x7fffffff) % uint32(len(partitions)) 293 return partitions[idx] 294 } 295 296 // Go port of the Java library's murmur2 function. 297 // https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L353 298 func murmur2(data []byte) uint32 { 299 length := len(data) 300 const ( 301 seed uint32 = 0x9747b28c 302 // 'm' and 'r' are mixing constants generated offline. 303 // They're not really 'magic', they just happen to work well. 304 m = 0x5bd1e995 305 r = 24 306 ) 307 308 // Initialize the hash to a random value 309 h := seed ^ uint32(length) 310 length4 := length / 4 311 312 for i := 0; i < length4; i++ { 313 i4 := i * 4 314 k := (uint32(data[i4+0]) & 0xff) + ((uint32(data[i4+1]) & 0xff) << 8) + ((uint32(data[i4+2]) & 0xff) << 16) + ((uint32(data[i4+3]) & 0xff) << 24) 315 k *= m 316 k ^= k >> r 317 k *= m 318 h *= m 319 h ^= k 320 } 321 322 // Handle the last few bytes of the input array 323 extra := length % 4 324 if extra >= 3 { 325 h ^= (uint32(data[(length & ^3)+2]) & 0xff) << 16 326 } 327 if extra >= 2 { 328 h ^= (uint32(data[(length & ^3)+1]) & 0xff) << 8 329 } 330 if extra >= 1 { 331 h ^= uint32(data[length & ^3]) & 0xff 332 h *= m 333 } 334 335 h ^= h >> 13 336 h *= m 337 h ^= h >> 15 338 339 return h 340 }