balancer.go 9.0 KB

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