setup follower by publisher

* the subscriber would getOrGen a local partition and wait
* the publisher would getOrGen a local partition. If localPartition follower is not setup, and init message has follower info, it would create followers based on init.Messages.
This commit is contained in:
chrislu
2024-03-24 12:57:09 -07:00
parent ac13644fc9
commit 205d2285f3
9 changed files with 339 additions and 434 deletions

View File

@@ -7,7 +7,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
"google.golang.org/grpc/peer"
"io"
"math/rand"
@@ -41,7 +40,6 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
// 3. write to the filer
var localTopicPartition *topic.LocalPartition
var isGenerated bool
req, err := stream.Recv()
if err != nil {
return err
@@ -58,7 +56,13 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
// get or generate a local partition
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
localTopicPartition, isGenerated, err = b.GetOrGenLocalPartition(t, p)
conf, readConfErr := b.readTopicConfFromFiler(t)
if readConfErr != nil {
response.Error = fmt.Sprintf("topic %v not found: %v", initMessage.Topic, readConfErr)
glog.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr)
return stream.Send(response)
}
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, p, conf)
if err != nil {
response.Error = fmt.Sprintf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
glog.Errorf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
@@ -67,25 +71,23 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
ackInterval = int(initMessage.AckInterval)
// connect to follower brokers
var followerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
var grpcConnection *grpc.ClientConn
if isGenerated && len(initMessage.FollowerBrokers) > 0 {
if localTopicPartition.FollowerStream == nil && len(initMessage.FollowerBrokers) > 0 {
follower := initMessage.FollowerBrokers[0]
ctx := stream.Context()
grpcConnection, err = pb.GrpcDial(ctx, follower, true, b.grpcDialOption)
localTopicPartition.GrpcConnection, err = pb.GrpcDial(ctx, follower, true, b.grpcDialOption)
if err != nil {
response.Error = fmt.Sprintf("fail to dial %s: %v", follower, err)
glog.Errorf("fail to dial %s: %v", follower, err)
return stream.Send(response)
}
followerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
followerStream, err = followerClient.PublishFollowMe(ctx)
followerClient := mq_pb.NewSeaweedMessagingClient(localTopicPartition.GrpcConnection)
localTopicPartition.FollowerStream, err = followerClient.PublishFollowMe(ctx)
if err != nil {
response.Error = fmt.Sprintf("fail to create publish client: %v", err)
glog.Errorf("fail to create publish client: %v", err)
return stream.Send(response)
}
if err = followerStream.Send(&mq_pb.PublishFollowMeRequest{
if err = localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Init{
Init: &mq_pb.PublishFollowMeRequest_InitMessage{
Topic: initMessage.Topic,
@@ -104,7 +106,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
ackCounter := 0
var ackSequence int64
defer func() {
if followerStream == nil {
if localTopicPartition.FollowerStream == nil {
// remove the publisher
localTopicPartition.Publishers.RemovePublisher(clientName)
glog.V(0).Infof("topic %v partition %v published %d messges Publisher:%d Subscriber:%d", initMessage.Topic, initMessage.Partition, ackSequence, localTopicPartition.Publishers.Size(), localTopicPartition.Subscribers.Size())
@@ -114,7 +116,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
}
}()
if followerStream != nil {
if localTopicPartition.FollowerStream != nil {
go func() {
defer func() {
println("stop receiving ack from follower")
@@ -126,11 +128,11 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
b.localTopicManager.RemoveTopicPartition(t, p)
}
println("closing grpcConnection to follower")
grpcConnection.Close()
localTopicPartition.GrpcConnection.Close()
}()
for {
ack, err := followerStream.Recv()
ack, err := localTopicPartition.FollowerStream.Recv()
if err != nil {
glog.Errorf("Error receiving response: %v", err)
return
@@ -153,7 +155,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
var receivedSequence, acknowledgedSequence int64
defer func() {
if followerStream != nil {
if localTopicPartition.FollowerStream != nil {
//if err := followerStream.CloseSend(); err != nil {
// glog.Errorf("Error closing follower stream: %v", err)
//}
@@ -193,9 +195,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
receivedSequence = dataMessage.TsNs
// maybe send to the follower
if followerStream != nil {
if localTopicPartition.FollowerStream != nil {
println("recv", string(dataMessage.Key), dataMessage.TsNs)
if followErr := followerStream.Send(&mq_pb.PublishFollowMeRequest{
if followErr := localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Data{
Data: dataMessage,
},
@@ -218,9 +220,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
}
}
if followerStream != nil {
if localTopicPartition.FollowerStream != nil {
// send close to the follower
if followErr := followerStream.Send(&mq_pb.PublishFollowMeRequest{
if followErr := localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Close{
Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
},

View File

@@ -16,40 +16,28 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
ctx := stream.Context()
clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
initMessage := req.GetInit()
if initMessage == nil {
glog.Errorf("missing init message")
return fmt.Errorf("missing init message")
}
t := topic.FromPbTopic(req.GetInit().Topic)
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
waitIntervalCount := 0
// get or generate a local partition
var localTopicPartition *topic.LocalPartition
for localTopicPartition == nil {
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition)
if err != nil {
glog.V(1).Infof("topic %v partition %v not setup", t, partition)
}
if localTopicPartition != nil {
break
}
waitIntervalCount++
if waitIntervalCount > 10 {
waitIntervalCount = 10
}
time.Sleep(time.Duration(waitIntervalCount) * 337 * time.Millisecond)
// Check if the client has disconnected by monitoring the context
select {
case <-ctx.Done():
err := ctx.Err()
if err == context.Canceled {
// Client disconnected
return nil
}
glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
return nil
default:
// Continue processing the request
}
conf, readConfErr := b.readTopicConfFromFiler(t)
if readConfErr != nil {
glog.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr)
return fmt.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr)
}
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition, conf)
if err != nil {
glog.Errorf("topic %v partition %v not setup", initMessage.Topic, partition)
return fmt.Errorf("topic %v partition %v not setup", initMessage.Topic, partition)
}
localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())

View File

@@ -39,18 +39,11 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
go func() {
// try to load the partition assignment from filer
if conf, err := b.readTopicConfFromFiler(topic.FromPbTopic(initMessage.Topic)); err == nil {
assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(conf.BrokerPartitionAssignments))
for i, assignment := range conf.BrokerPartitionAssignments {
assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{
Partition: assignment.Partition,
Broker: assignment.LeaderBroker,
}
}
// send partition assignment to subscriber
cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
AssignedPartitions: assignedPartitions,
PartitionAssignments: conf.BrokerPartitionAssignments,
},
},
}

View File

@@ -56,12 +56,12 @@ func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb.
return conf, nil
}
func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
b.accessLock.Lock()
defer b.accessLock.Unlock()
if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil {
localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition)
localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition, conf)
if err != nil {
return nil, false, err
}
@@ -69,12 +69,8 @@ func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition top
return localPartition, isGenerated, nil
}
func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
self := b.option.BrokerAddress()
conf, err := b.readTopicConfFromFiler(t)
if err != nil {
return nil, isGenerated, err
}
for _, assignment := range conf.BrokerPartitionAssignments {
if assignment.LeaderBroker == string(self) && partition.Equals(topic.FromPbPartition(assignment.Partition)) {
localPartition = topic.FromPbBrokerPartitionAssignment(b.option.BrokerAddress(), partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))

View File

@@ -91,26 +91,26 @@ func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCo
var wg sync.WaitGroup
semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit)
for _, assigned := range assignment.AssignedPartitions {
for _, assigned := range assignment.PartitionAssignments {
wg.Add(1)
semaphore <- struct{}{}
go func(partition *mq_pb.Partition, broker string) {
go func(assigned *mq_pb.BrokerPartitionAssignment) {
defer wg.Done()
defer func() { <-semaphore }()
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
err := sub.onEachPartition(partition, broker)
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
err := sub.onEachPartition(assigned)
if err != nil {
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err)
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
}
}(assigned.Partition, assigned.Broker)
}(assigned)
}
wg.Wait()
}
func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker string) error {
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment) error {
// connect to the partition broker
return pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
subscribeClient, err := client.SubscribeMessage(context.Background(), &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Init{
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
@@ -118,11 +118,12 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
PartitionOffset: &mq_pb.PartitionOffset{
Partition: partition,
Partition: assigned.Partition,
StartTsNs: sub.alreadyProcessedTsNs,
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
},
Filter: sub.ContentConfig.Filter,
FollowerBrokers: assigned.FollowerBrokers,
},
},
})
@@ -131,7 +132,7 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
return fmt.Errorf("create subscribe client: %v", err)
}
glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
if sub.OnCompletionFunc != nil {
defer sub.OnCompletionFunc()

View File

@@ -103,22 +103,22 @@ func (cg *ConsumerGroup) RebalanceConsumberGroupInstances(knownPartitionSlotToBr
partitionSlots = make([]*PartitionSlotToConsumerInstance, 0)
}
consumerGroupInstance.Partitions = ToPartitions(partitionSlotToBrokerList.RingSize, partitionSlots)
assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(partitionSlots))
assignedPartitions := make([]*mq_pb.BrokerPartitionAssignment, len(partitionSlots))
for i, partitionSlot := range partitionSlots {
assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{
assignedPartitions[i] = &mq_pb.BrokerPartitionAssignment{
Partition: &mq_pb.Partition{
RangeStop: partitionSlot.RangeStop,
RangeStart: partitionSlot.RangeStart,
RingSize: partitionSlotToBrokerList.RingSize,
UnixTimeNs: partitionSlot.UnixTimeNs,
},
Broker: partitionSlot.Broker,
LeaderBroker: partitionSlot.Broker,
}
}
response := &mq_pb.SubscriberToSubCoordinatorResponse{
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
AssignedPartitions: assignedPartitions,
PartitionAssignments: assignedPartitions,
},
},
}

View File

@@ -6,6 +6,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"google.golang.org/grpc"
"sync"
"sync/atomic"
"time"
@@ -26,6 +27,9 @@ type LocalPartition struct {
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
FollowerId int32
FollowerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
GrpcConnection *grpc.ClientConn
}
var TIME_FORMAT = "2006-01-02-15-04-05"