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