seaweedfs/weed/mq/topic/local_partition.go

240 lines
6.8 KiB
Go
Raw Normal View History

package topic
import (
2024-03-28 13:58:20 +08:00
"context"
2023-08-29 00:02:12 +08:00
"fmt"
2024-01-15 16:20:12 +08:00
"github.com/seaweedfs/seaweedfs/weed/glog"
"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"
2024-03-31 15:19:16 +08:00
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
2024-03-17 14:16:33 +08:00
"sync"
2024-03-11 05:34:28 +08:00
"sync/atomic"
"time"
)
type LocalPartition struct {
2024-03-17 14:16:33 +08:00
ListenersWaits int64
AckTsNs int64
2024-03-17 14:16:33 +08:00
// notifying clients
ListenersLock sync.Mutex
ListenersCond *sync.Cond
Partition
2024-03-17 01:51:47 +08:00
LogBuffer *log_buffer.LogBuffer
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
2024-03-28 14:11:35 +08:00
followerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
followerGrpcConnection *grpc.ClientConn
2024-03-28 14:10:24 +08:00
follower string
}
var TIME_FORMAT = "2006-01-02-15-04-05"
2024-03-01 01:38:52 +08:00
2024-03-28 14:42:19 +08:00
func NewLocalPartition(partition Partition, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
2024-03-17 14:16:33 +08:00
lp := &LocalPartition{
2023-08-29 00:02:12 +08:00
Partition: partition,
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
2023-12-12 04:05:54 +08:00
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
2023-08-29 00:02:12 +08:00
}
2024-03-17 14:16:33 +08:00
lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
2*time.Minute, logFlushFn, readFromDiskFn, func() {
if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
lp.ListenersCond.Broadcast()
}
})
return lp
2023-08-29 00:02:12 +08:00
}
2024-03-28 14:10:24 +08:00
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
p.LogBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
2024-03-28 14:10:24 +08:00
// maybe send to the follower
2024-03-28 14:11:35 +08:00
if p.followerStream != nil {
2024-04-03 06:31:31 +08:00
// println("recv", string(message.Key), message.TsNs)
2024-03-28 14:11:35 +08:00
if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
2024-03-28 14:10:24 +08:00
Message: &mq_pb.PublishFollowMeRequest_Data{
Data: message,
},
}); followErr != nil {
return fmt.Errorf("send to follower %s: %v", p.follower, followErr)
}
} else {
atomic.StoreInt64(&p.AckTsNs, message.TsNs)
}
return nil
}
2024-01-15 16:20:12 +08:00
func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
var processedPosition log_buffer.MessagePosition
var readPersistedLogErr error
var readInMemoryLogErr error
var isDone bool
for {
processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
2024-01-15 16:20:12 +08:00
if readPersistedLogErr != nil {
glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
return readPersistedLogErr
}
if isDone {
return nil
}
startPosition = processedPosition
processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
2024-03-11 05:34:28 +08:00
if isDone {
return nil
}
2024-01-15 16:20:12 +08:00
startPosition = processedPosition
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
continue
}
if readInMemoryLogErr != nil {
glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
return readInMemoryLogErr
}
}
2024-01-08 16:03:08 +08:00
}
func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
return p.LogBuffer.GetEarliestTime()
2024-01-08 16:03:08 +08:00
}
func (p *LocalPartition) HasData() bool {
return !p.LogBuffer.GetEarliestTime().IsZero()
2024-01-08 16:03:08 +08:00
}
func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
return p.LogBuffer.GetEarliestPosition()
2023-08-28 04:13:14 +08:00
}
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
2023-12-12 04:05:54 +08:00
func (p *LocalPartition) closePublishers() {
p.Publishers.SignalShutdown()
}
func (p *LocalPartition) closeSubscribers() {
p.Subscribers.SignalShutdown()
}
func (p *LocalPartition) WaitUntilNoPublishers() {
for {
2024-03-31 11:37:57 +08:00
if p.Publishers.Size() == 0 {
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
2023-12-12 04:05:54 +08:00
return
}
time.Sleep(113 * time.Millisecond)
}
}
2024-03-28 13:58:20 +08:00
func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
2024-03-28 14:11:35 +08:00
if p.followerStream != nil {
2024-03-28 13:58:20 +08:00
return nil
}
if len(initMessage.FollowerBrokers) == 0 {
return nil
}
2024-03-28 14:10:24 +08:00
p.follower = initMessage.FollowerBrokers[0]
2024-03-28 13:58:20 +08:00
ctx := context.Background()
2024-03-28 14:11:35 +08:00
p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.follower, true, grpcDialOption)
2024-03-28 13:58:20 +08:00
if err != nil {
2024-03-28 14:10:24 +08:00
return fmt.Errorf("fail to dial %s: %v", p.follower, err)
2024-03-28 13:58:20 +08:00
}
2024-03-28 14:11:35 +08:00
followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
p.followerStream, err = followerClient.PublishFollowMe(ctx)
2024-03-28 13:58:20 +08:00
if err != nil {
return fmt.Errorf("fail to create publish client: %v", err)
}
2024-03-28 14:11:35 +08:00
if err = p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
2024-03-28 13:58:20 +08:00
Message: &mq_pb.PublishFollowMeRequest_Init{
Init: &mq_pb.PublishFollowMeRequest_InitMessage{
Topic: initMessage.Topic,
Partition: initMessage.Partition,
},
},
}); err != nil {
return err
}
// start receiving ack from follower
go func() {
defer func() {
2024-04-03 06:31:31 +08:00
// println("stop receiving ack from follower")
2024-03-28 13:58:20 +08:00
}()
for {
2024-03-28 14:11:35 +08:00
ack, err := p.followerStream.Recv()
2024-03-28 13:58:20 +08:00
if err != nil {
2024-03-31 15:19:16 +08:00
e, _ := status.FromError(err)
if e.Code() == codes.Canceled {
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)
2024-03-28 13:58:20 +08:00
return
}
atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
2024-04-03 06:31:31 +08:00
// println("recv ack", ack.AckTsNs)
2024-03-28 13:58:20 +08:00
}
}()
return nil
}
func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
2024-03-28 13:48:37 +08:00
if p.Publishers.Size() == 0 && p.Subscribers.Size() == 0 {
p.LogBuffer.ShutdownLogBuffer()
for !p.LogBuffer.IsAllFlushed() {
time.Sleep(113 * time.Millisecond)
}
2024-03-28 14:55:06 +08:00
if p.followerStream != nil {
// send close to the follower
if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Close{
Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
},
}); followErr != nil {
glog.Errorf("Error closing follower stream: %v", followErr)
}
glog.V(4).Infof("closing grpcConnection to follower")
p.followerGrpcConnection.Close()
p.followerStream = nil
2024-03-31 11:37:57 +08:00
p.follower = ""
2024-03-28 14:02:51 +08:00
}
2024-03-28 14:55:06 +08:00
hasShutdown = true
}
2024-03-28 14:02:51 +08:00
2024-03-31 11:37:46 +08:00
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
}
2024-03-11 05:34:28 +08:00
func (p *LocalPartition) Shutdown() {
p.closePublishers()
p.closeSubscribers()
p.LogBuffer.ShutdownLogBuffer()
2024-03-18 13:24:20 +08:00
glog.V(0).Infof("local partition %v shutting down", p.Partition)
2024-03-11 05:34:28 +08:00
}
func (p *LocalPartition) NotifyLogFlushed(flushTsNs int64) {
if p.followerStream != nil {
if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Flush{
Flush: &mq_pb.PublishFollowMeRequest_FlushMessage{
TsNs: flushTsNs,
},
},
}); followErr != nil {
glog.Errorf("send follower %s flush message: %v", p.follower, followErr)
}
2024-04-03 06:31:31 +08:00
// println("notifying", p.follower, "flushed at", flushTsNs)
}
}