1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-05-17 17:02:31 +02:00
This commit is contained in:
chrislu 2024-03-16 10:51:47 -07:00
parent 059df3957d
commit 529fa8c4c8
8 changed files with 62 additions and 63 deletions

View file

@ -31,9 +31,9 @@ func NewLockClient(grpcDialOption grpc.DialOption, seedFiler pb.ServerAddress) *
type LiveLock struct { type LiveLock struct {
key string key string
renewToken string renewToken string
expireAtNs int64 expireAtNs int64
hostFiler pb.ServerAddress hostFiler pb.ServerAddress
cancelCh chan struct{} cancelCh chan struct{}
grpcDialOption grpc.DialOption grpcDialOption grpc.DialOption
isLocked bool isLocked bool
self string self string

View file

@ -63,9 +63,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
for _, follower := range initMessage.FollowerBrokers { for _, follower := range initMessage.FollowerBrokers {
followErr := b.withBrokerClient(false, pb.ServerAddress(follower), func(client mq_pb.SeaweedMessagingClient) error { followErr := b.withBrokerClient(false, pb.ServerAddress(follower), func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.PublishFollowMe(context.Background(), &mq_pb.PublishFollowMeRequest{ _, err := client.PublishFollowMe(context.Background(), &mq_pb.PublishFollowMeRequest{
Topic: initMessage.Topic, Topic: initMessage.Topic,
Partition: initMessage.Partition, Partition: initMessage.Partition,
BrokerSelf: string(b.option.BrokerAddress()), BrokerSelf: string(b.option.BrokerAddress()),
}) })
return err return err
}) })

View file

@ -12,7 +12,7 @@ import (
"time" "time"
) )
func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error){ func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error) {
glog.V(0).Infof("PublishFollowMe %v", request) glog.V(0).Infof("PublishFollowMe %v", request)
var wg sync.WaitGroup var wg sync.WaitGroup
wg.Add(1) wg.Add(1)
@ -75,7 +75,7 @@ func (b *MessageQueueBroker) doFollowInMemoryMessage(c context.Context, client m
} }
if resp.Message != nil { if resp.Message != nil {
// process ctrl message or data message // process ctrl message or data message
switch m:= resp.Message.(type) { switch m := resp.Message.(type) {
case *mq_pb.FollowInMemoryMessagesResponse_Data: case *mq_pb.FollowInMemoryMessagesResponse_Data:
// process data message // process data message
print("d") print("d")

View file

@ -130,7 +130,7 @@ func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer
return return
} }
func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMessagesRequest, stream mq_pb.SeaweedMessaging_FollowInMemoryMessagesServer) (err error) { func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMessagesRequest, stream mq_pb.SeaweedMessaging_FollowInMemoryMessagesServer) (err error) {
ctx := stream.Context() ctx := stream.Context()
clientName := req.GetInit().ConsumerId clientName := req.GetInit().ConsumerId
@ -188,8 +188,7 @@ func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMe
// to indicate the follower is connected // to indicate the follower is connected
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{ Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{},
},
}, },
}) })
@ -200,7 +199,7 @@ func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMe
var prevFlushTsNs int64 var prevFlushTsNs int64
_,_, err = localTopicPartition.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, func() bool { _, _, err = localTopicPartition.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, func() bool {
if !isConnected { if !isConnected {
return false return false
} }
@ -285,12 +284,12 @@ func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMe
// send the log entry // send the log entry
if err := stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ if err := stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Data{ Message: &mq_pb.FollowInMemoryMessagesResponse_Data{
Data: &mq_pb.DataMessage{ Data: &mq_pb.DataMessage{
Key: logEntry.Key, Key: logEntry.Key,
Value: logEntry.Data, Value: logEntry.Data,
TsNs: logEntry.TsNs, TsNs: logEntry.TsNs,
}, },
}}); err != nil { }}); err != nil {
glog.Errorf("Error sending setup response: %v", err) glog.Errorf("Error sending setup response: %v", err)
return false, err return false, err
} }

View file

@ -102,7 +102,7 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
assignment.LeaderBroker = "" assignment.LeaderBroker = ""
count++ count++
} }
for i:=0; i<followerCount; i++ { for i := 0; i < followerCount; i++ {
if i >= len(assignment.FollowerBrokers) { if i >= len(assignment.FollowerBrokers) {
count++ count++
continue continue
@ -128,7 +128,7 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
hasEmptyFollowers := false hasEmptyFollowers := false
j := 0 j := 0
for ; j<len(assignment.FollowerBrokers); j++ { for ; j < len(assignment.FollowerBrokers); j++ {
if assignment.FollowerBrokers[j] == "" { if assignment.FollowerBrokers[j] == "" {
hasChanges = true hasChanges = true
if i < len(pickedBrokers) { if i < len(pickedBrokers) {

View file

@ -86,19 +86,19 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
singleActiveBroker := cmap.New[*BrokerStats]() singleActiveBroker := cmap.New[*BrokerStats]()
singleActiveBroker.SetIfAbsent("localhost:1", &BrokerStats{}) singleActiveBroker.SetIfAbsent("localhost:1", &BrokerStats{})
tests := []struct { tests := []struct {
name string name string
args args args args
hasChanges bool hasChanges bool
}{ }{
{ {
name: "test empty leader", name: "test empty leader",
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 1, followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "", LeaderBroker: "",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:2", "localhost:2",
}, },
@ -112,10 +112,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 1, followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"", "",
}, },
@ -129,10 +129,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 1, followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:200", "localhost:200",
}, },
@ -146,10 +146,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 1, followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:100", LeaderBroker: "localhost:100",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:200", "localhost:200",
}, },
@ -163,10 +163,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 3, followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:2", "localhost:2",
}, },
@ -180,10 +180,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: activeBrokers, activeBrokers: activeBrokers,
followerCount: 10, followerCount: 10,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:2", "localhost:2",
}, },
@ -197,10 +197,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: lowActiveBrokers, activeBrokers: lowActiveBrokers,
followerCount: 3, followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:2", "localhost:2",
}, },
@ -214,10 +214,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: lowActiveBrokers, activeBrokers: lowActiveBrokers,
followerCount: 1, followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
}, },
}, },
}, },
@ -228,10 +228,10 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
args: args{ args: args{
activeBrokers: singleActiveBroker, activeBrokers: singleActiveBroker,
followerCount: 3, followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{ assignments: []*mq_pb.BrokerPartitionAssignment{
{ {
LeaderBroker: "localhost:1", LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{}, Partition: &mq_pb.Partition{},
FollowerBrokers: []string{ FollowerBrokers: []string{
"localhost:2", "localhost:2",
}, },

View file

@ -12,13 +12,13 @@ import (
type LocalPartition struct { type LocalPartition struct {
Partition Partition
isLeader bool isLeader bool
FollowerBrokers []pb.ServerAddress FollowerBrokers []pb.ServerAddress
LogBuffer *log_buffer.LogBuffer LogBuffer *log_buffer.LogBuffer
ConsumerCount int32 ConsumerCount int32
Publishers *LocalPartitionPublishers Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers Subscribers *LocalPartitionSubscribers
FollowerId int32 FollowerId int32
} }
var TIME_FORMAT = "2006-01-02-15-04-05" var TIME_FORMAT = "2006-01-02-15-04-05"

View file

@ -27,24 +27,24 @@ type LogFlushFuncType func(logBuffer *LogBuffer, startTime, stopTime time.Time,
type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error) type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error)
type LogBuffer struct { type LogBuffer struct {
LastFlushTsNs int64 LastFlushTsNs int64
name string name string
prevBuffers *SealedBuffers prevBuffers *SealedBuffers
buf []byte buf []byte
batchIndex int64 batchIndex int64
idx []int idx []int
pos int pos int
startTime time.Time startTime time.Time
stopTime time.Time stopTime time.Time
lastFlushDataTime time.Time lastFlushDataTime time.Time
sizeBuf []byte sizeBuf []byte
flushInterval time.Duration flushInterval time.Duration
flushFn LogFlushFuncType flushFn LogFlushFuncType
ReadFromDiskFn LogReadFromDiskFuncType ReadFromDiskFn LogReadFromDiskFuncType
notifyFn func() notifyFn func()
isStopping *atomic.Bool isStopping *atomic.Bool
flushChan chan *dataToFlush flushChan chan *dataToFlush
lastTsNs int64 lastTsNs int64
sync.RWMutex sync.RWMutex
} }