2023-08-28 04:13:14 +08:00
|
|
|
package broker
|
|
|
|
|
|
|
|
import (
|
2023-12-12 04:05:54 +08:00
|
|
|
"context"
|
2023-08-28 08:50:59 +08:00
|
|
|
"fmt"
|
2023-08-28 04:13:14 +08:00
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
2024-05-19 15:46:12 +08:00
|
|
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
2023-08-28 04:13:14 +08:00
|
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
2024-01-08 16:03:08 +08:00
|
|
|
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
2024-05-17 11:28:19 +08:00
|
|
|
"io"
|
2023-08-28 04:13:14 +08:00
|
|
|
"time"
|
|
|
|
)
|
|
|
|
|
2024-05-17 11:28:19 +08:00
|
|
|
func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_SubscribeMessageServer) error {
|
2023-08-28 04:13:14 +08:00
|
|
|
|
2024-05-17 11:28:19 +08:00
|
|
|
req, err := stream.Recv()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if req.GetInit() == nil {
|
2024-03-25 03:57:09 +08:00
|
|
|
glog.Errorf("missing init message")
|
|
|
|
return fmt.Errorf("missing init message")
|
|
|
|
}
|
|
|
|
|
2024-05-17 11:28:19 +08:00
|
|
|
ctx := stream.Context()
|
|
|
|
clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
|
|
|
|
|
2023-12-12 04:05:54 +08:00
|
|
|
t := topic.FromPbTopic(req.GetInit().Topic)
|
2024-01-06 07:24:14 +08:00
|
|
|
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
|
2024-01-12 15:08:02 +08:00
|
|
|
|
|
|
|
glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
|
|
|
|
|
2024-03-25 04:04:59 +08:00
|
|
|
localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, partition)
|
|
|
|
if getOrGenErr != nil {
|
|
|
|
return getOrGenErr
|
2023-08-28 04:13:14 +08:00
|
|
|
}
|
|
|
|
|
2023-12-12 04:05:54 +08:00
|
|
|
localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())
|
|
|
|
glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
|
|
|
|
isConnected := true
|
|
|
|
sleepIntervalCount := 0
|
2024-01-15 16:20:12 +08:00
|
|
|
|
|
|
|
var counter int64
|
2023-12-12 04:05:54 +08:00
|
|
|
defer func() {
|
|
|
|
isConnected = false
|
|
|
|
localTopicPartition.Subscribers.RemoveSubscriber(clientName)
|
2024-01-15 16:20:12 +08:00
|
|
|
glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
|
2024-01-17 00:43:07 +08:00
|
|
|
if localTopicPartition.MaybeShutdownLocalPartition() {
|
2024-04-02 06:02:52 +08:00
|
|
|
b.localTopicManager.RemoveLocalPartition(t, partition)
|
2024-01-17 00:43:07 +08:00
|
|
|
}
|
2023-12-12 04:05:54 +08:00
|
|
|
}()
|
|
|
|
|
2024-05-19 15:46:12 +08:00
|
|
|
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)
|
2023-12-12 04:05:54 +08:00
|
|
|
}
|
|
|
|
|
2024-05-17 11:28:19 +08:00
|
|
|
go func() {
|
2024-05-19 15:46:12 +08:00
|
|
|
var lastOffset int64
|
2024-05-17 11:28:19 +08:00
|
|
|
for {
|
|
|
|
ack, err := stream.Recv()
|
|
|
|
if err != nil {
|
|
|
|
if err == io.EOF {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
glog.V(0).Infof("topic %v partition %v subscriber %s error: %v", t, partition, clientName, err)
|
|
|
|
break
|
|
|
|
}
|
2024-05-19 15:46:12 +08:00
|
|
|
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)
|
|
|
|
}
|
|
|
|
}
|
2024-05-17 11:28:19 +08:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2024-01-15 16:20:12 +08:00
|
|
|
return localTopicPartition.Subscribe(clientName, startPosition, func() bool {
|
2023-12-12 04:05:54 +08:00
|
|
|
if !isConnected {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
sleepIntervalCount++
|
2024-03-11 05:34:28 +08:00
|
|
|
if sleepIntervalCount > 32 {
|
|
|
|
sleepIntervalCount = 32
|
2023-12-12 04:05:54 +08:00
|
|
|
}
|
2024-03-11 05:34:28 +08:00
|
|
|
time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond)
|
2023-12-12 04:05:54 +08:00
|
|
|
|
|
|
|
// Check if the client has disconnected by monitoring the context
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
err := ctx.Err()
|
|
|
|
if err == context.Canceled {
|
|
|
|
// Client disconnected
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
|
|
|
|
return false
|
|
|
|
default:
|
|
|
|
// Continue processing the request
|
|
|
|
}
|
|
|
|
|
|
|
|
return true
|
2024-03-08 02:50:09 +08:00
|
|
|
}, func(logEntry *filer_pb.LogEntry) (bool, error) {
|
2023-12-12 04:05:54 +08:00
|
|
|
// reset the sleep interval count
|
|
|
|
sleepIntervalCount = 0
|
2023-09-01 15:36:51 +08:00
|
|
|
|
2024-01-06 07:14:25 +08:00
|
|
|
if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
|
2023-08-28 04:13:14 +08:00
|
|
|
Data: &mq_pb.DataMessage{
|
2024-03-08 02:53:30 +08:00
|
|
|
Key: logEntry.Key,
|
|
|
|
Value: logEntry.Data,
|
2023-12-12 04:05:54 +08:00
|
|
|
TsNs: logEntry.TsNs,
|
2023-08-28 04:13:14 +08:00
|
|
|
},
|
2024-03-11 05:34:28 +08:00
|
|
|
}}); err != nil {
|
|
|
|
glog.Errorf("Error sending data: %v", err)
|
|
|
|
return false, err
|
|
|
|
}
|
|
|
|
|
|
|
|
counter++
|
|
|
|
return false, nil
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2024-05-19 15:46:12 +08:00
|
|
|
func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (startPosition log_buffer.MessagePosition) {
|
|
|
|
if initMessage == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
offset := initMessage.GetPartitionOffset()
|
2024-03-11 05:34:28 +08:00
|
|
|
if offset.StartTsNs != 0 {
|
|
|
|
startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
|
2024-05-19 15:46:12 +08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil{
|
|
|
|
startPosition = log_buffer.NewMessagePosition(storedOffset, -2)
|
|
|
|
return
|
2024-03-11 05:34:28 +08:00
|
|
|
}
|
2024-05-19 15:46:12 +08:00
|
|
|
|
2024-03-11 05:34:28 +08:00
|
|
|
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
|
|
|
|
startPosition = log_buffer.NewMessagePosition(1, -3)
|
|
|
|
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
|
|
|
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
|
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|