Merge accumulated changes related to message queue (#5098)
* balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * tracking topic=>broker * merge * comment
This commit is contained in:
41
weed/mq/sub_coordinator/consumer_group.go
Normal file
41
weed/mq/sub_coordinator/consumer_group.go
Normal file
@@ -0,0 +1,41 @@
|
||||
package sub_coordinator
|
||||
|
||||
import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
)
|
||||
type ConsumerGroupInstance struct {
|
||||
InstanceId string
|
||||
// the consumer group instance may not have an active partition
|
||||
Partitions []*topic.Partition
|
||||
ResponseChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||
}
|
||||
type ConsumerGroup struct {
|
||||
// map a consumer group instance id to a consumer group instance
|
||||
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance]
|
||||
mapping *PartitionConsumerMapping
|
||||
}
|
||||
|
||||
func NewConsumerGroup() *ConsumerGroup {
|
||||
return &ConsumerGroup{
|
||||
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
|
||||
mapping: NewPartitionConsumerMapping(pub_balancer.MaxPartitionCount),
|
||||
}
|
||||
}
|
||||
|
||||
func NewConsumerGroupInstance(instanceId string) *ConsumerGroupInstance {
|
||||
return &ConsumerGroupInstance{
|
||||
InstanceId: instanceId,
|
||||
ResponseChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1),
|
||||
}
|
||||
}
|
||||
func (cg *ConsumerGroup) OnAddConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
|
||||
}
|
||||
func (cg *ConsumerGroup) OnRemoveConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
|
||||
|
||||
}
|
||||
func (cg *ConsumerGroup) OnPartitionListChange() {
|
||||
|
||||
}
|
||||
86
weed/mq/sub_coordinator/coordinator.go
Normal file
86
weed/mq/sub_coordinator/coordinator.go
Normal file
@@ -0,0 +1,86 @@
|
||||
package sub_coordinator
|
||||
|
||||
import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
)
|
||||
|
||||
|
||||
type TopicConsumerGroups struct {
|
||||
// map a consumer group name to a consumer group
|
||||
ConsumerGroups cmap.ConcurrentMap[string, *ConsumerGroup]
|
||||
}
|
||||
|
||||
// Coordinator coordinates the instances in the consumer group for one topic.
|
||||
// It is responsible for:
|
||||
// 1. (Maybe) assigning partitions when a consumer instance is up/down.
|
||||
|
||||
type Coordinator struct {
|
||||
// map topic name to consumer groups
|
||||
TopicSubscribers cmap.ConcurrentMap[string, *TopicConsumerGroups]
|
||||
balancer *pub_balancer.Balancer
|
||||
}
|
||||
|
||||
func NewCoordinator(balancer *pub_balancer.Balancer) *Coordinator {
|
||||
return &Coordinator{
|
||||
TopicSubscribers: cmap.New[*TopicConsumerGroups](),
|
||||
balancer: balancer,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Coordinator) GetTopicConsumerGroups(topic *mq_pb.Topic) *TopicConsumerGroups {
|
||||
topicName := toTopicName(topic)
|
||||
tcg, _ := c.TopicSubscribers.Get(topicName)
|
||||
if tcg == nil {
|
||||
tcg = &TopicConsumerGroups{
|
||||
ConsumerGroups: cmap.New[*ConsumerGroup](),
|
||||
}
|
||||
c.TopicSubscribers.Set(topicName, tcg)
|
||||
}
|
||||
return tcg
|
||||
}
|
||||
func (c *Coordinator) RemoveTopic(topic *mq_pb.Topic) {
|
||||
topicName := toTopicName(topic)
|
||||
c.TopicSubscribers.Remove(topicName)
|
||||
}
|
||||
|
||||
func toTopicName(topic *mq_pb.Topic) string {
|
||||
topicName := topic.Namespace + "." + topic.Name
|
||||
return topicName
|
||||
}
|
||||
|
||||
func (c *Coordinator) AddSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) *ConsumerGroupInstance{
|
||||
tcg := c.GetTopicConsumerGroups(topic)
|
||||
cg, _ := tcg.ConsumerGroups.Get(consumerGroup)
|
||||
if cg == nil {
|
||||
cg = NewConsumerGroup()
|
||||
tcg.ConsumerGroups.Set(consumerGroup, cg)
|
||||
}
|
||||
cgi, _ := cg.ConsumerGroupInstances.Get(consumerGroupInstance)
|
||||
if cgi == nil {
|
||||
cgi = NewConsumerGroupInstance(consumerGroupInstance)
|
||||
cg.ConsumerGroupInstances.Set(consumerGroupInstance, cgi)
|
||||
}
|
||||
cg.OnAddConsumerGroupInstance(consumerGroupInstance, topic)
|
||||
return cgi
|
||||
}
|
||||
|
||||
func (c *Coordinator) RemoveSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) {
|
||||
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
|
||||
if tcg == nil {
|
||||
return
|
||||
}
|
||||
cg, _ := tcg.ConsumerGroups.Get(consumerGroup)
|
||||
if cg == nil {
|
||||
return
|
||||
}
|
||||
cg.ConsumerGroupInstances.Remove(consumerGroupInstance)
|
||||
cg.OnRemoveConsumerGroupInstance(consumerGroupInstance, topic)
|
||||
if cg.ConsumerGroupInstances.Count() == 0 {
|
||||
tcg.ConsumerGroups.Remove(consumerGroup)
|
||||
}
|
||||
if tcg.ConsumerGroups.Count() == 0 {
|
||||
c.RemoveTopic(topic)
|
||||
}
|
||||
}
|
||||
119
weed/mq/sub_coordinator/partition_consumer_mapping.go
Normal file
119
weed/mq/sub_coordinator/partition_consumer_mapping.go
Normal file
@@ -0,0 +1,119 @@
|
||||
package sub_coordinator
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"time"
|
||||
)
|
||||
|
||||
type PartitionConsumerMapping struct {
|
||||
currentMapping *PartitionSlotToConsumerInstanceList
|
||||
prevMappings []*PartitionSlotToConsumerInstanceList
|
||||
}
|
||||
|
||||
func NewPartitionConsumerMapping(ringSize int32) *PartitionConsumerMapping {
|
||||
newVersion := time.Now().UnixNano()
|
||||
return &PartitionConsumerMapping{
|
||||
currentMapping: NewPartitionSlotToConsumerInstanceList(ringSize, newVersion),
|
||||
}
|
||||
}
|
||||
|
||||
// Balance goal:
|
||||
// 1. max processing power utilization
|
||||
// 2. allow one consumer instance to be down unexpectedly
|
||||
// without affecting the processing power utilization
|
||||
|
||||
func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitions []*topic.Partition, consumerInstanceIds []string) {
|
||||
if len(partitions) == 0 || len(consumerInstanceIds) == 0 {
|
||||
return
|
||||
}
|
||||
newVersion := time.Now().UnixNano()
|
||||
newMapping := NewPartitionSlotToConsumerInstanceList(partitions[0].RingSize, newVersion)
|
||||
newMapping.PartitionSlots = doBalanceSticky(partitions, consumerInstanceIds, pcm.prevMappings[0])
|
||||
if pcm.currentMapping != nil {
|
||||
pcm.prevMappings = append(pcm.prevMappings, pcm.currentMapping)
|
||||
}
|
||||
pcm.currentMapping = newMapping
|
||||
}
|
||||
|
||||
func doBalanceSticky(partitions []*topic.Partition, consumerInstanceIds []string, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) {
|
||||
// collect previous consumer instance ids
|
||||
prevConsumerInstanceIds := make(map[string]struct{})
|
||||
if prevMapping != nil {
|
||||
for _, prevPartitionSlot := range prevMapping.PartitionSlots {
|
||||
if prevPartitionSlot.AssignedInstanceId != "" {
|
||||
prevConsumerInstanceIds[prevPartitionSlot.AssignedInstanceId] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
// collect current consumer instance ids
|
||||
currConsumerInstanceIds := make(map[string]struct{})
|
||||
for _, consumerInstanceId := range consumerInstanceIds {
|
||||
currConsumerInstanceIds[consumerInstanceId] = struct{}{}
|
||||
}
|
||||
|
||||
// check deleted consumer instances
|
||||
deletedConsumerInstanceIds := make(map[string]struct{})
|
||||
for consumerInstanceId := range prevConsumerInstanceIds {
|
||||
if _, ok := currConsumerInstanceIds[consumerInstanceId]; !ok {
|
||||
deletedConsumerInstanceIds[consumerInstanceId] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
// convert partition slots from list to a map
|
||||
prevPartitionSlotMap := make(map[string]*PartitionSlotToConsumerInstance)
|
||||
if prevMapping != nil {
|
||||
for _, partitionSlot := range prevMapping.PartitionSlots {
|
||||
key := fmt.Sprintf("%d-%d", partitionSlot.RangeStart, partitionSlot.RangeStop)
|
||||
prevPartitionSlotMap[key] = partitionSlot
|
||||
}
|
||||
}
|
||||
|
||||
// make a copy of old mapping, skipping the deleted consumer instances
|
||||
newPartitionSlots := ToPartitionSlots(partitions)
|
||||
for _, newPartitionSlot := range newPartitionSlots {
|
||||
key := fmt.Sprintf("%d-%d", newPartitionSlot.RangeStart, newPartitionSlot.RangeStop)
|
||||
if prevPartitionSlot, ok := prevPartitionSlotMap[key]; ok {
|
||||
if _, ok := deletedConsumerInstanceIds[prevPartitionSlot.AssignedInstanceId]; !ok {
|
||||
newPartitionSlot.AssignedInstanceId = prevPartitionSlot.AssignedInstanceId
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// for all consumer instances, count the average number of partitions
|
||||
// that are assigned to them
|
||||
consumerInstancePartitionCount := make(map[string]int)
|
||||
for _, newPartitionSlot := range newPartitionSlots {
|
||||
if newPartitionSlot.AssignedInstanceId != "" {
|
||||
consumerInstancePartitionCount[newPartitionSlot.AssignedInstanceId]++
|
||||
}
|
||||
}
|
||||
// average number of partitions that are assigned to each consumer instance
|
||||
averageConsumerInstanceLoad := float32(len(partitions)) / float32(len(consumerInstanceIds))
|
||||
|
||||
// assign unassigned partition slots to consumer instances that is underloaded
|
||||
consumerInstanceIdsIndex := 0
|
||||
for _, newPartitionSlot := range newPartitionSlots {
|
||||
if newPartitionSlot.AssignedInstanceId == "" {
|
||||
for avoidDeadLoop := len(consumerInstanceIds); avoidDeadLoop > 0; avoidDeadLoop-- {
|
||||
consumerInstanceId := consumerInstanceIds[consumerInstanceIdsIndex]
|
||||
if float32(consumerInstancePartitionCount[consumerInstanceId]) < averageConsumerInstanceLoad {
|
||||
newPartitionSlot.AssignedInstanceId = consumerInstanceId
|
||||
consumerInstancePartitionCount[consumerInstanceId]++
|
||||
consumerInstanceIdsIndex++
|
||||
if consumerInstanceIdsIndex >= len(consumerInstanceIds) {
|
||||
consumerInstanceIdsIndex = 0
|
||||
}
|
||||
break
|
||||
} else {
|
||||
consumerInstanceIdsIndex++
|
||||
if consumerInstanceIdsIndex >= len(consumerInstanceIds) {
|
||||
consumerInstanceIdsIndex = 0
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return newPartitionSlots
|
||||
}
|
||||
312
weed/mq/sub_coordinator/partition_consumer_mapping_test.go
Normal file
312
weed/mq/sub_coordinator/partition_consumer_mapping_test.go
Normal file
@@ -0,0 +1,312 @@
|
||||
package sub_coordinator
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func Test_doBalanceSticky(t *testing.T) {
|
||||
type args struct {
|
||||
partitions []*topic.Partition
|
||||
consumerInstanceIds []string
|
||||
prevMapping *PartitionSlotToConsumerInstanceList
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
wantPartitionSlots []*PartitionSlotToConsumerInstance
|
||||
}{
|
||||
{
|
||||
name: "1 consumer instance, 1 partition",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1"},
|
||||
prevMapping: nil,
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 1 partition",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
|
||||
prevMapping: nil,
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "1 consumer instance, 2 partitions",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1"},
|
||||
prevMapping: nil,
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 2 partitions",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
|
||||
prevMapping: nil,
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 2 partitions, 1 deleted consumer instance",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
|
||||
prevMapping: &PartitionSlotToConsumerInstanceList{
|
||||
PartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-3",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 2 partitions, 1 new consumer instance",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"},
|
||||
prevMapping: &PartitionSlotToConsumerInstanceList{
|
||||
PartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-3",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-3",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 2 partitions, 1 new partition",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
{
|
||||
RangeStart: 100,
|
||||
RangeStop: 150,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
|
||||
prevMapping: &PartitionSlotToConsumerInstanceList{
|
||||
PartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
{
|
||||
RangeStart: 100,
|
||||
RangeStop: 150,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "2 consumer instances, 2 partitions, 1 new partition, 1 new consumer instance",
|
||||
args: args{
|
||||
partitions: []*topic.Partition{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
},
|
||||
{
|
||||
RangeStart: 100,
|
||||
RangeStop: 150,
|
||||
},
|
||||
},
|
||||
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"},
|
||||
prevMapping: &PartitionSlotToConsumerInstanceList{
|
||||
PartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
|
||||
{
|
||||
RangeStart: 0,
|
||||
RangeStop: 50,
|
||||
AssignedInstanceId: "consumer-instance-1",
|
||||
},
|
||||
{
|
||||
RangeStart: 50,
|
||||
RangeStop: 100,
|
||||
AssignedInstanceId: "consumer-instance-2",
|
||||
},
|
||||
{
|
||||
RangeStart: 100,
|
||||
RangeStop: 150,
|
||||
AssignedInstanceId: "consumer-instance-3",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
if gotPartitionSlots := doBalanceSticky(tt.args.partitions, tt.args.consumerInstanceIds, tt.args.prevMapping); !reflect.DeepEqual(gotPartitionSlots, tt.wantPartitionSlots) {
|
||||
t.Errorf("doBalanceSticky() = %v, want %v", gotPartitionSlots, tt.wantPartitionSlots)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
32
weed/mq/sub_coordinator/partition_list.go
Normal file
32
weed/mq/sub_coordinator/partition_list.go
Normal file
@@ -0,0 +1,32 @@
|
||||
package sub_coordinator
|
||||
|
||||
import "github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
|
||||
type PartitionSlotToConsumerInstance struct {
|
||||
RangeStart int32
|
||||
RangeStop int32
|
||||
AssignedInstanceId string
|
||||
}
|
||||
|
||||
type PartitionSlotToConsumerInstanceList struct {
|
||||
PartitionSlots []*PartitionSlotToConsumerInstance
|
||||
RingSize int32
|
||||
Version int64
|
||||
}
|
||||
|
||||
func NewPartitionSlotToConsumerInstanceList(ringSize int32, version int64) *PartitionSlotToConsumerInstanceList {
|
||||
return &PartitionSlotToConsumerInstanceList{
|
||||
RingSize: ringSize,
|
||||
Version: version,
|
||||
}
|
||||
}
|
||||
|
||||
func ToPartitionSlots(partitions []*topic.Partition) (partitionSlots []*PartitionSlotToConsumerInstance) {
|
||||
for _, partition := range partitions {
|
||||
partitionSlots = append(partitionSlots, &PartitionSlotToConsumerInstance{
|
||||
RangeStart: partition.RangeStart,
|
||||
RangeStop: partition.RangeStop,
|
||||
})
|
||||
}
|
||||
return
|
||||
}
|
||||
Reference in New Issue
Block a user