github.com/deanMdreon/kafka-go@v0.4.32/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  type randomBalancer struct {
   171  	mock int // mocked return value, used for testing
   172  }
   173  
   174  func (b randomBalancer) Balance(msg Message, partitions ...int) (partition int) {
   175  	if b.mock != 0 {
   176  		return b.mock
   177  	}
   178  	return partitions[rand.Int()%len(partitions)]
   179  }
   180  
   181  // CRC32Balancer is a Balancer that uses the CRC32 hash function to determine
   182  // which partition to route messages to.  This ensures that messages with the
   183  // same key are routed to the same partition.  This balancer is compatible with
   184  // the built-in hash partitioners in librdkafka and the language bindings that
   185  // are built on top of it, including the
   186  // github.com/confluentinc/confluent-kafka-go Go package.
   187  //
   188  // With the Consistent field false (default), this partitioner is equivalent to
   189  // the "consistent_random" setting in librdkafka.  When Consistent is true, this
   190  // partitioner is equivalent to the "consistent" setting.  The latter will hash
   191  // empty or nil keys into the same partition.
   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  type CRC32Balancer struct {
   197  	Consistent bool
   198  	random     randomBalancer
   199  }
   200  
   201  func (b CRC32Balancer) Balance(msg Message, partitions ...int) (partition int) {
   202  	// NOTE: the crc32 balancers in librdkafka don't differentiate between nil
   203  	//       and empty keys.  both cases are treated as unset.
   204  	if len(msg.Key) == 0 && !b.Consistent {
   205  		return b.random.Balance(msg, partitions...)
   206  	}
   207  
   208  	idx := crc32.ChecksumIEEE(msg.Key) % uint32(len(partitions))
   209  	return partitions[idx]
   210  }
   211  
   212  // Murmur2Balancer is a Balancer that uses the Murmur2 hash function to
   213  // determine which partition to route messages to.  This ensures that messages
   214  // with the same key are routed to the same partition.  This balancer is
   215  // compatible with the partitioner used by the Java library and by librdkafka's
   216  // "murmur2" and "murmur2_random" partitioners. /
   217  //
   218  // With the Consistent field false (default), this partitioner is equivalent to
   219  // the "murmur2_random" setting in librdkafka.  When Consistent is true, this
   220  // partitioner is equivalent to the "murmur2" setting.  The latter will hash
   221  // nil keys into the same partition.  Empty, non-nil keys are always hashed to
   222  // the same partition regardless of configuration.
   223  //
   224  // Unless you are absolutely certain that all your messages will have keys, it's
   225  // best to leave the Consistent flag off.  Otherwise, you run the risk of
   226  // creating a very hot partition.
   227  //
   228  // Note that the librdkafka documentation states that the "murmur2_random" is
   229  // functionally equivalent to the default Java partitioner.  That's because the
   230  // Java partitioner will use a round robin balancer instead of random on nil
   231  // keys.  We choose librdkafka's implementation because it arguably has a larger
   232  // install base.
   233  type Murmur2Balancer struct {
   234  	Consistent bool
   235  	random     randomBalancer
   236  }
   237  
   238  func (b Murmur2Balancer) Balance(msg Message, partitions ...int) (partition int) {
   239  	// NOTE: the murmur2 balancers in java and librdkafka treat a nil key as
   240  	//       non-existent while treating an empty slice as a defined value.
   241  	if msg.Key == nil && !b.Consistent {
   242  		return b.random.Balance(msg, partitions...)
   243  	}
   244  
   245  	idx := (murmur2(msg.Key) & 0x7fffffff) % uint32(len(partitions))
   246  	return partitions[idx]
   247  }
   248  
   249  // Go port of the Java library's murmur2 function.
   250  // https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L353
   251  func murmur2(data []byte) uint32 {
   252  	length := len(data)
   253  	const (
   254  		seed uint32 = 0x9747b28c
   255  		// 'm' and 'r' are mixing constants generated offline.
   256  		// They're not really 'magic', they just happen to work well.
   257  		m = 0x5bd1e995
   258  		r = 24
   259  	)
   260  
   261  	// Initialize the hash to a random value
   262  	h := seed ^ uint32(length)
   263  	length4 := length / 4
   264  
   265  	for i := 0; i < length4; i++ {
   266  		i4 := i * 4
   267  		k := (uint32(data[i4+0]) & 0xff) + ((uint32(data[i4+1]) & 0xff) << 8) + ((uint32(data[i4+2]) & 0xff) << 16) + ((uint32(data[i4+3]) & 0xff) << 24)
   268  		k *= m
   269  		k ^= k >> r
   270  		k *= m
   271  		h *= m
   272  		h ^= k
   273  	}
   274  
   275  	// Handle the last few bytes of the input array
   276  	extra := length % 4
   277  	if extra >= 3 {
   278  		h ^= (uint32(data[(length & ^3)+2]) & 0xff) << 16
   279  	}
   280  	if extra >= 2 {
   281  		h ^= (uint32(data[(length & ^3)+1]) & 0xff) << 8
   282  	}
   283  	if extra >= 1 {
   284  		h ^= uint32(data[length & ^3]) & 0xff
   285  		h *= m
   286  	}
   287  
   288  	h ^= h >> 13
   289  	h *= m
   290  	h ^= h >> 15
   291  
   292  	return h
   293  }