persist consumer group offset
1. use one follower 2. read write consumer group offset
This commit is contained in:
@@ -36,7 +36,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
|
||||
// if is leader, notify the followers to drain existing topic partition subscriptions
|
||||
if request.IsLeader {
|
||||
for _, brokerPartition := range request.BrokerPartitionAssignments {
|
||||
for _, follower := range brokerPartition.FollowerBrokers {
|
||||
if follower := brokerPartition.FollowerBroker; follower != "" {
|
||||
err := pb.WithBrokerGrpcClient(false, follower, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
_, err := client.AssignTopicPartitions(context.Background(), request)
|
||||
return err
|
||||
|
||||
@@ -5,6 +5,7 @@ import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||
@@ -51,12 +52,42 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
||||
}
|
||||
}()
|
||||
|
||||
var startPosition log_buffer.MessagePosition
|
||||
if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil {
|
||||
startPosition = getRequestPosition(req.GetInit().GetPartitionOffset())
|
||||
startPosition := b.getRequestPosition(req.GetInit())
|
||||
|
||||
// connect to the follower
|
||||
var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
|
||||
glog.V(0).Infof("follower broker: %v", req.GetInit().FollowerBroker)
|
||||
if req.GetInit().FollowerBroker != "" {
|
||||
follower := req.GetInit().FollowerBroker
|
||||
if followerGrpcConnection, err := pb.GrpcDial(ctx, follower, true, b.grpcDialOption); err != nil {
|
||||
return fmt.Errorf("fail to dial %s: %v", follower, err)
|
||||
} else {
|
||||
defer func() {
|
||||
println("closing SubscribeFollowMe connection", follower)
|
||||
followerGrpcConnection.Close()
|
||||
}()
|
||||
followerClient := mq_pb.NewSeaweedMessagingClient(followerGrpcConnection)
|
||||
if subscribeFollowMeStream, err = followerClient.SubscribeFollowMe(ctx); err != nil {
|
||||
return fmt.Errorf("fail to subscribe to %s: %v", follower, err)
|
||||
} else {
|
||||
if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
|
||||
Message: &mq_pb.SubscribeFollowMeRequest_Init{
|
||||
Init: &mq_pb.SubscribeFollowMeRequest_InitMessage{
|
||||
Topic: req.GetInit().Topic,
|
||||
Partition: req.GetInit().GetPartitionOffset().Partition,
|
||||
ConsumerGroup: req.GetInit().ConsumerGroup,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
return fmt.Errorf("fail to send init to %s: %v", follower, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
glog.V(0).Infof("follower %s connected", follower)
|
||||
}
|
||||
|
||||
go func() {
|
||||
var lastOffset int64
|
||||
for {
|
||||
ack, err := stream.Recv()
|
||||
if err != nil {
|
||||
@@ -66,7 +97,34 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
||||
glog.V(0).Infof("topic %v partition %v subscriber %s error: %v", t, partition, clientName, err)
|
||||
break
|
||||
}
|
||||
println(clientName, "ack =>", ack.GetAck().Sequence)
|
||||
lastOffset = ack.GetAck().Sequence
|
||||
if subscribeFollowMeStream != nil {
|
||||
if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
|
||||
Message: &mq_pb.SubscribeFollowMeRequest_Ack{
|
||||
Ack: &mq_pb.SubscribeFollowMeRequest_AckMessage{
|
||||
TsNs: lastOffset,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
glog.Errorf("Error sending ack to follower: %v", err)
|
||||
break
|
||||
}
|
||||
println("forwarding ack", lastOffset)
|
||||
}
|
||||
}
|
||||
if lastOffset > 0 {
|
||||
if err := b.saveConsumerGroupOffset(t, partition, req.GetInit().ConsumerGroup, lastOffset); err != nil {
|
||||
glog.Errorf("saveConsumerGroupOffset: %v", err)
|
||||
}
|
||||
if subscribeFollowMeStream != nil {
|
||||
if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
|
||||
Message: &mq_pb.SubscribeFollowMeRequest_Close{
|
||||
Close: &mq_pb.SubscribeFollowMeRequest_CloseMessage{},
|
||||
},
|
||||
}); err != nil {
|
||||
glog.Errorf("Error sending close to follower: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
@@ -115,10 +173,20 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
||||
})
|
||||
}
|
||||
|
||||
func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer.MessagePosition) {
|
||||
func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (startPosition log_buffer.MessagePosition) {
|
||||
if initMessage == nil {
|
||||
return
|
||||
}
|
||||
offset := initMessage.GetPartitionOffset()
|
||||
if offset.StartTsNs != 0 {
|
||||
startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
|
||||
return
|
||||
}
|
||||
if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil{
|
||||
startPosition = log_buffer.NewMessagePosition(storedOffset, -2)
|
||||
return
|
||||
}
|
||||
|
||||
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
|
||||
startPosition = log_buffer.NewMessagePosition(1, -3)
|
||||
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
||||
|
||||
@@ -25,11 +25,7 @@ func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_Sub
|
||||
}
|
||||
|
||||
// create an in-memory offset
|
||||
subscriberProgress := &SubscriberProgress{
|
||||
Topic: topic.FromPbTopic(initMessage.Topic),
|
||||
Partition: topic.FromPbPartition(initMessage.Partition),
|
||||
ConsumerGroup: "consumer_group",
|
||||
}
|
||||
var lastOffset int64
|
||||
|
||||
// follow each published messages
|
||||
for {
|
||||
@@ -46,8 +42,8 @@ func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_Sub
|
||||
|
||||
// Process the received message
|
||||
if ackMessage := req.GetAck(); ackMessage != nil {
|
||||
subscriberProgress.Offset = ackMessage.TsNs
|
||||
println("offset", subscriberProgress.Offset)
|
||||
lastOffset = ackMessage.TsNs
|
||||
println("offset", lastOffset)
|
||||
} else if closeMessage := req.GetClose(); closeMessage != nil {
|
||||
glog.V(0).Infof("topic %v partition %v subscribe stream closed: %v", initMessage.Topic, initMessage.Partition, closeMessage)
|
||||
return nil
|
||||
@@ -58,19 +54,47 @@ func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_Sub
|
||||
|
||||
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
|
||||
|
||||
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
|
||||
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
|
||||
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
|
||||
offsetFileName := fmt.Sprintf("%s.offset", subscriberProgress.ConsumerGroup)
|
||||
err = b.saveConsumerGroupOffset(t, p, initMessage.ConsumerGroup, lastOffset)
|
||||
|
||||
offsetBytes := make([]byte, 8)
|
||||
util.Uint64toBytes(offsetBytes, uint64(subscriberProgress.Offset))
|
||||
|
||||
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
|
||||
return filer.SaveInsideFiler(client, partitionDir, offsetFileName, offsetBytes)
|
||||
})
|
||||
|
||||
glog.V(0).Infof("shut down follower for %v %v", t, p)
|
||||
glog.V(0).Infof("shut down follower for %v", initMessage)
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func (b *MessageQueueBroker) readConsumerGroupOffset(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (offset int64, err error) {
|
||||
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.PartitionOffset.Partition)
|
||||
|
||||
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
|
||||
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
|
||||
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
|
||||
offsetFileName := fmt.Sprintf("%s.offset", initMessage.ConsumerGroup)
|
||||
|
||||
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
|
||||
data, err := filer.ReadInsideFiler(client, partitionDir, offsetFileName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if len(data) != 8 {
|
||||
return fmt.Errorf("no offset found")
|
||||
}
|
||||
offset = int64(util.BytesToUint64(data))
|
||||
return nil
|
||||
})
|
||||
return offset, err
|
||||
}
|
||||
|
||||
func (b *MessageQueueBroker) saveConsumerGroupOffset(t topic.Topic, p topic.Partition, consumerGroup string, offset int64) error {
|
||||
|
||||
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
|
||||
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
|
||||
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
|
||||
offsetFileName := fmt.Sprintf("%s.offset", consumerGroup)
|
||||
|
||||
offsetBytes := make([]byte, 8)
|
||||
util.Uint64toBytes(offsetBytes, uint64(offset))
|
||||
|
||||
return b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
|
||||
glog.V(0).Infof("saving topic %s partition %v consumer group %s offset %d", t, p, consumerGroup, offset)
|
||||
return filer.SaveInsideFiler(client, partitionDir, offsetFileName, offsetBytes)
|
||||
})
|
||||
}
|
||||
|
||||
@@ -103,7 +103,7 @@ func (b *MessageQueueBroker) genLogOnDiskReadFunc(t topic.Topic, partition *mq_p
|
||||
|
||||
eachFileFn := func(entry *filer_pb.Entry, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
|
||||
if len(entry.Content) > 0 {
|
||||
glog.Warningf("this should not happen. unexpected content in %s/%s", partitionDir, entry.Name)
|
||||
// skip .offset files
|
||||
return
|
||||
}
|
||||
var urlStrings []string
|
||||
|
||||
@@ -1,10 +0,0 @@
|
||||
package broker
|
||||
|
||||
import "github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
|
||||
type SubscriberProgress struct {
|
||||
topic.Topic
|
||||
topic.Partition
|
||||
ConsumerGroup string
|
||||
Offset int64
|
||||
}
|
||||
@@ -145,7 +145,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro
|
||||
Topic: p.config.Topic.ToPbTopic(),
|
||||
Partition: job.Partition,
|
||||
AckInterval: 128,
|
||||
FollowerBrokers: job.FollowerBrokers,
|
||||
FollowerBroker: job.FollowerBroker,
|
||||
PublisherName: p.config.PublisherName,
|
||||
},
|
||||
},
|
||||
|
||||
@@ -102,11 +102,10 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: &mq_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: sub.alreadyProcessedTsNs,
|
||||
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
},
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBrokers: assigned.FollowerBrokers,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
},
|
||||
},
|
||||
});err != nil {
|
||||
@@ -154,7 +153,6 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
if processErr != nil {
|
||||
return fmt.Errorf("process error: %v", processErr)
|
||||
}
|
||||
sub.alreadyProcessedTsNs = m.Data.TsNs
|
||||
partitionOffsetChan <- m.Data.TsNs
|
||||
if !shouldContinue {
|
||||
return nil
|
||||
|
||||
@@ -37,7 +37,6 @@ type TopicSubscriber struct {
|
||||
OnCompletionFunc OnCompletionFunc
|
||||
bootstrapBrokers []string
|
||||
waitForMoreMessage bool
|
||||
alreadyProcessedTsNs int64
|
||||
activeProcessors map[topic.Partition]*ProcessorState
|
||||
activeProcessorsLock sync.Mutex
|
||||
}
|
||||
@@ -50,7 +49,6 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
|
||||
brokerPartitionAssignmentChan: make(chan *mq_pb.BrokerPartitionAssignment, 1024),
|
||||
bootstrapBrokers: bootstrapBrokers,
|
||||
waitForMoreMessage: true,
|
||||
alreadyProcessedTsNs: content.StartTime.UnixNano(),
|
||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -48,20 +48,11 @@ func pickBrokers(brokers cmap.ConcurrentMap[string, *BrokerStats], count int32)
|
||||
return pickedBrokers
|
||||
}
|
||||
|
||||
// reservoir sampling select N brokers from the active brokers, with exclusion of the excluded brokers
|
||||
func pickBrokersExcluded(brokers []string, count int, excludedLeadBroker string, excludedBrokers []string) []string {
|
||||
// convert the excluded brokers to a map
|
||||
excludedBrokerMap := make(map[string]bool)
|
||||
for _, broker := range excludedBrokers {
|
||||
excludedBrokerMap[broker] = true
|
||||
}
|
||||
if excludedLeadBroker != "" {
|
||||
excludedBrokerMap[excludedLeadBroker] = true
|
||||
}
|
||||
|
||||
// reservoir sampling select N brokers from the active brokers, with exclusion of the excluded broker
|
||||
func pickBrokersExcluded(brokers []string, count int, excludedLeadBroker string, excludedBroker string) []string {
|
||||
pickedBrokers := make([]string, 0, count)
|
||||
for i, broker := range brokers {
|
||||
if _, found := excludedBrokerMap[broker]; found {
|
||||
if broker == excludedBroker {
|
||||
continue
|
||||
}
|
||||
if len(pickedBrokers) < count {
|
||||
@@ -102,21 +93,15 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
|
||||
assignment.LeaderBroker = ""
|
||||
count++
|
||||
}
|
||||
for i := 0; i < followerCount; i++ {
|
||||
if i >= len(assignment.FollowerBrokers) {
|
||||
count++
|
||||
continue
|
||||
}
|
||||
if assignment.FollowerBrokers[i] == "" {
|
||||
count++
|
||||
} else if _, found := activeBrokers.Get(assignment.FollowerBrokers[i]); !found {
|
||||
assignment.FollowerBrokers[i] = ""
|
||||
count++
|
||||
}
|
||||
if assignment.FollowerBroker == "" {
|
||||
count++
|
||||
} else if _, found := activeBrokers.Get(assignment.FollowerBroker); !found {
|
||||
assignment.FollowerBroker = ""
|
||||
count++
|
||||
}
|
||||
|
||||
if count > 0 {
|
||||
pickedBrokers := pickBrokersExcluded(candidates, count, assignment.LeaderBroker, assignment.FollowerBrokers)
|
||||
pickedBrokers := pickBrokersExcluded(candidates, count, assignment.LeaderBroker, assignment.FollowerBroker)
|
||||
i := 0
|
||||
if assignment.LeaderBroker == "" {
|
||||
if i < len(pickedBrokers) {
|
||||
@@ -125,34 +110,13 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
|
||||
hasChanges = true
|
||||
}
|
||||
}
|
||||
|
||||
hasEmptyFollowers := false
|
||||
j := 0
|
||||
for ; j < len(assignment.FollowerBrokers); j++ {
|
||||
if assignment.FollowerBrokers[j] == "" {
|
||||
if assignment.FollowerBroker == "" {
|
||||
if i < len(pickedBrokers) {
|
||||
assignment.FollowerBroker = pickedBrokers[i]
|
||||
i++
|
||||
hasChanges = true
|
||||
if i < len(pickedBrokers) {
|
||||
assignment.FollowerBrokers[j] = pickedBrokers[i]
|
||||
i++
|
||||
} else {
|
||||
hasEmptyFollowers = true
|
||||
}
|
||||
}
|
||||
}
|
||||
if hasEmptyFollowers {
|
||||
var followerBrokers []string
|
||||
for _, follower := range assignment.FollowerBrokers {
|
||||
if follower != "" {
|
||||
followerBrokers = append(followerBrokers, follower)
|
||||
}
|
||||
}
|
||||
assignment.FollowerBrokers = followerBrokers
|
||||
}
|
||||
|
||||
if i < len(pickedBrokers) {
|
||||
assignment.FollowerBrokers = append(assignment.FollowerBrokers, pickedBrokers[i:]...)
|
||||
hasChanges = true
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -92,7 +92,7 @@ func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *Broke
|
||||
partitionSlotToBrokerList, _ = balancer.TopicToBrokers.Get(topicKey)
|
||||
}
|
||||
}
|
||||
partitionSlotToBrokerList.AddBroker(partition, broker)
|
||||
partitionSlotToBrokerList.AddBroker(partition, broker, topicPartitionStats.Follower)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -10,6 +10,7 @@ type PartitionSlotToBroker struct {
|
||||
RangeStop int32
|
||||
UnixTimeNs int64
|
||||
AssignedBroker string
|
||||
FollowerBroker string
|
||||
}
|
||||
|
||||
type PartitionSlotToBrokerList struct {
|
||||
@@ -23,16 +24,18 @@ func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList {
|
||||
}
|
||||
}
|
||||
|
||||
func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string) {
|
||||
func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string, follower string) {
|
||||
for _, partitionSlot := range ps.PartitionSlots {
|
||||
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
|
||||
if partitionSlot.AssignedBroker == broker {
|
||||
return
|
||||
}
|
||||
if partitionSlot.AssignedBroker != "" {
|
||||
if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {
|
||||
glog.V(0).Infof("partition %s broker change: %s => %s", partition, partitionSlot.AssignedBroker, broker)
|
||||
partitionSlot.AssignedBroker = broker
|
||||
}
|
||||
partitionSlot.AssignedBroker = broker
|
||||
if partitionSlot.FollowerBroker != "" && partitionSlot.FollowerBroker != follower {
|
||||
glog.V(0).Infof("partition %s follower change: %s => %s", partition, partitionSlot.FollowerBroker, follower)
|
||||
partitionSlot.FollowerBroker = follower
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -41,6 +44,7 @@ func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broke
|
||||
RangeStop: partition.RangeStop,
|
||||
UnixTimeNs: partition.UnixTimeNs,
|
||||
AssignedBroker: broker,
|
||||
FollowerBroker: follower,
|
||||
})
|
||||
}
|
||||
func (ps *PartitionSlotToBrokerList) RemoveBroker(broker string) {
|
||||
|
||||
@@ -66,7 +66,7 @@ func (cg *ConsumerGroup) OnPartitionListChange(assignments []*mq_pb.BrokerPartit
|
||||
}
|
||||
partitionSlotToBrokerList := pub_balancer.NewPartitionSlotToBrokerList(pub_balancer.MaxPartitionCount)
|
||||
for _, assignment := range assignments {
|
||||
partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker)
|
||||
partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker, assignment.FollowerBroker)
|
||||
}
|
||||
cg.BalanceConsumerGroupInstances(partitionSlotToBrokerList, "partition list change")
|
||||
}
|
||||
@@ -80,7 +80,7 @@ func (cg *ConsumerGroup) BalanceConsumerGroupInstances(knownPartitionSlotToBroke
|
||||
if conf, err := cg.filerClientAccessor.ReadTopicConfFromFiler(cg.topic); err == nil {
|
||||
partitionSlotToBrokerList = pub_balancer.NewPartitionSlotToBrokerList(pub_balancer.MaxPartitionCount)
|
||||
for _, assignment := range conf.BrokerPartitionAssignments {
|
||||
partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker)
|
||||
partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker, assignment.FollowerBroker)
|
||||
}
|
||||
} else {
|
||||
glog.V(0).Infof("fail to read topic conf from filer: %v", err)
|
||||
@@ -118,7 +118,8 @@ func (cg *ConsumerGroup) BalanceConsumerGroupInstances(knownPartitionSlotToBroke
|
||||
RingSize: partitionSlotToBrokerList.RingSize,
|
||||
UnixTimeNs: partitionSlot.UnixTimeNs,
|
||||
},
|
||||
LeaderBroker: partitionSlot.Broker,
|
||||
LeaderBroker: partitionSlot.Broker,
|
||||
FollowerBroker: partitionSlot.FollowerBroker,
|
||||
}
|
||||
}
|
||||
response := &mq_pb.SubscriberToSubCoordinatorResponse{
|
||||
|
||||
@@ -86,6 +86,7 @@ func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerI
|
||||
RangeStop: partition.RangeStop,
|
||||
UnixTimeNs: partition.UnixTimeNs,
|
||||
Broker: partition.AssignedBroker,
|
||||
FollowerBroker: partition.FollowerBroker,
|
||||
})
|
||||
}
|
||||
for _, newPartitionSlot := range newPartitionSlots {
|
||||
|
||||
@@ -8,6 +8,7 @@ type PartitionSlotToConsumerInstance struct {
|
||||
UnixTimeNs int64
|
||||
Broker string
|
||||
AssignedInstanceId string
|
||||
FollowerBroker string
|
||||
}
|
||||
|
||||
type PartitionSlotToConsumerInstanceList struct {
|
||||
|
||||
@@ -99,6 +99,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
|
||||
Partition: localPartition.Partition.ToPbPartition(),
|
||||
PublisherCount: int32(localPartition.Publishers.Size()),
|
||||
SubscriberCount: int32(localPartition.Subscribers.Size()),
|
||||
Follower: localPartition.Follower,
|
||||
}
|
||||
// fmt.Printf("collect topic %+v partition %+v\n", topicPartition, localPartition.Partition)
|
||||
}
|
||||
|
||||
@@ -30,7 +30,7 @@ type LocalPartition struct {
|
||||
|
||||
publishFolloweMeStream mq_pb.SeaweedMessaging_PublishFollowMeClient
|
||||
followerGrpcConnection *grpc.ClientConn
|
||||
follower string
|
||||
Follower string
|
||||
}
|
||||
|
||||
var TIME_FORMAT = "2006-01-02-15-04-05"
|
||||
@@ -62,7 +62,7 @@ func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
|
||||
Data: message,
|
||||
},
|
||||
}); followErr != nil {
|
||||
return fmt.Errorf("send to follower %s: %v", p.follower, followErr)
|
||||
return fmt.Errorf("send to follower %s: %v", p.Follower, followErr)
|
||||
}
|
||||
} else {
|
||||
atomic.StoreInt64(&p.AckTsNs, message.TsNs)
|
||||
@@ -137,15 +137,15 @@ func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessa
|
||||
if p.publishFolloweMeStream != nil {
|
||||
return nil
|
||||
}
|
||||
if len(initMessage.FollowerBrokers) == 0 {
|
||||
if initMessage.FollowerBroker == "" {
|
||||
return nil
|
||||
}
|
||||
|
||||
p.follower = initMessage.FollowerBrokers[0]
|
||||
p.Follower = initMessage.FollowerBroker
|
||||
ctx := context.Background()
|
||||
p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.follower, true, grpcDialOption)
|
||||
p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.Follower, true, grpcDialOption)
|
||||
if err != nil {
|
||||
return fmt.Errorf("fail to dial %s: %v", p.follower, err)
|
||||
return fmt.Errorf("fail to dial %s: %v", p.Follower, err)
|
||||
}
|
||||
followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
|
||||
p.publishFolloweMeStream, err = followerClient.PublishFollowMe(ctx)
|
||||
@@ -174,10 +174,10 @@ func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessa
|
||||
if err != nil {
|
||||
e, _ := status.FromError(err)
|
||||
if e.Code() == codes.Canceled {
|
||||
glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.follower)
|
||||
glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.Follower)
|
||||
return
|
||||
}
|
||||
glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.follower, err)
|
||||
glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.Follower, err)
|
||||
return
|
||||
}
|
||||
atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
|
||||
@@ -206,13 +206,13 @@ func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
|
||||
glog.V(4).Infof("closing grpcConnection to follower")
|
||||
p.followerGrpcConnection.Close()
|
||||
p.publishFolloweMeStream = nil
|
||||
p.follower = ""
|
||||
p.Follower = ""
|
||||
}
|
||||
|
||||
hasShutdown = true
|
||||
}
|
||||
|
||||
glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.follower, hasShutdown)
|
||||
glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.Follower, hasShutdown)
|
||||
return
|
||||
}
|
||||
|
||||
@@ -232,8 +232,8 @@ func (p *LocalPartition) NotifyLogFlushed(flushTsNs int64) {
|
||||
},
|
||||
},
|
||||
}); followErr != nil {
|
||||
glog.Errorf("send follower %s flush message: %v", p.follower, followErr)
|
||||
glog.Errorf("send follower %s flush message: %v", p.Follower, followErr)
|
||||
}
|
||||
// println("notifying", p.follower, "flushed at", flushTsNs)
|
||||
// println("notifying", p.Follower, "flushed at", flushTsNs)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user