balancer.go 11 KB

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