Browse Source
Squashed commit of the following:
Squashed commit of the following:
commitpull/4778/head32f4b1a130
Author: chrislu <chris.lu@gmail.com> Date: Sun Aug 20 22:52:19 2023 -0700 fix compilation commite77ad33b7c
Author: chrislu <chris.lu@gmail.com> Date: Sun Aug 20 22:46:44 2023 -0700 pub commitf431f30cc7
Author: chrislu <chris.lu@gmail.com> Date: Sun Aug 20 13:27:39 2023 -0700 fix generic type commit4e9dcb1829
Merge: 30f942580 16e3f2d52 Author: chrislu <chris.lu@gmail.com> Date: Sun Aug 20 12:47:14 2023 -0700 Merge branch 'master' into pubsub commit 30f942580ad1bb32ae94aade2e3a21ec3ab63e21 Author: chrislu <chris.lu@gmail.com> Date: Sun Aug 20 11:10:58 2023 -0700 wip commit f8b00980bc2f3879bb43decffd9a08d842f196f2 Author: chrislu <chris.lu@gmail.com> Date: Tue Jul 25 09:14:35 2023 -0700 add design document commit 08d2bebe42a26ebc39f1542f54d99e73620727dd Author: chrislu <chris.lu@gmail.com> Date: Tue Jul 25 09:14:06 2023 -0700 minor commit bcfa7982b262a40fcdce6fc6613fad2ce07c13da Author: chrislu <chris.lu@gmail.com> Date: Tue Jul 25 09:13:49 2023 -0700 rename
chrislu
1 year ago
16 changed files with 1995 additions and 296 deletions
-
2go.mod
-
6go.sum
-
107weed/mq/broker/brokder_grpc_admin.go
-
122weed/mq/broker/brokder_grpc_pub.go
-
14weed/mq/broker/broker_segment_serde.go
-
21weed/mq/broker/broker_server.go
-
5weed/mq/segment/message_serde_test.go
-
54weed/mq/topic/local_manager.go
-
40weed/mq/topic/local_partition.go
-
29weed/mq/topic/local_topic.go
-
32weed/mq/topic/partition.go
-
21weed/mq/topic/topic.go
-
81weed/mq/topic_allocation/allocation.go
-
74weed/pb/mq.proto
-
1535weed/pb/mq_pb/mq.pb.go
-
148weed/pb/mq_pb/mq_grpc.pb.go
@ -1,16 +1,136 @@ |
|||
package broker |
|||
|
|||
import ( |
|||
"context" |
|||
"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/mq_pb" |
|||
) |
|||
|
|||
// For a new or re-configured topic, or one of the broker went offline,
|
|||
// the pub clients ask one broker what are the brokers for all the topic partitions.
|
|||
// The broker will lock the topic on write.
|
|||
// 1. if the topic is not found, create the topic, and allocate the topic partitions to the brokers
|
|||
// 2. if the topic is found, return the brokers for the topic partitions
|
|||
// For a topic to read from, the sub clients ask one broker what are the brokers for all the topic partitions.
|
|||
// The broker will lock the topic on read.
|
|||
// 1. if the topic is not found, return error
|
|||
// 2. if the topic is found, return the brokers for the topic partitions
|
|||
//
|
|||
// If the topic needs to be re-balanced, the admin client will lock the topic,
|
|||
// 1. collect throughput information for all the brokers
|
|||
// 2. adjust the topic partitions to the brokers
|
|||
// 3. notify the brokers to add/remove partitions to host
|
|||
// 3.1 When locking the topic, the partitions and brokers should be remembered in the lock.
|
|||
// 4. the brokers will stop process incoming messages if not the right partition
|
|||
// 4.1 the pub clients will need to re-partition the messages and publish to the right brokers for the partition3
|
|||
// 4.2 the sub clients will need to change the brokers to read from
|
|||
//
|
|||
// The following is from each individual component's perspective:
|
|||
// For a pub client
|
|||
// For current topic/partition, ask one broker for the brokers for the topic partitions
|
|||
// 1. connect to the brokers and keep sending, until the broker returns error, or the broker leader is moved.
|
|||
// For a sub client
|
|||
// For current topic/partition, ask one broker for the brokers for the topic partitions
|
|||
// 1. connect to the brokers and keep reading, until the broker returns error, or the broker leader is moved.
|
|||
// For a broker
|
|||
// Upon a pub client lookup:
|
|||
// 1. lock the topic
|
|||
// 2. if already has topic partition assignment, check all brokers are healthy
|
|||
// 3. if not, create topic partition assignment
|
|||
// 2. return the brokers for the topic partitions
|
|||
// 3. unlock the topic
|
|||
// Upon a sub client lookup:
|
|||
// 1. lock the topic
|
|||
// 2. if already has topic partition assignment, check all brokers are healthy
|
|||
// 3. if not, return error
|
|||
// 2. return the brokers for the topic partitions
|
|||
// 3. unlock the topic
|
|||
// For an admin tool
|
|||
// 0. collect stats from all the brokers, and find the topic worth moving
|
|||
// 1. lock the topic
|
|||
// 2. collect throughput information for all the brokers
|
|||
// 3. adjust the topic partitions to the brokers
|
|||
// 4. notify the brokers to add/remove partitions to host
|
|||
// 5. the brokers will stop process incoming messages if not the right partition
|
|||
// 6. unlock the topic
|
|||
|
|||
/* |
|||
The messages is buffered in memory, and saved to filer under |
|||
The messages are buffered in memory, and saved to filer under |
|||
/topics/<topic>/<date>/<hour>/<segment>/*.msg |
|||
/topics/<topic>/<date>/<hour>/segment |
|||
/topics/<topic>/info/segment_<id>.meta |
|||
|
|||
|
|||
|
|||
*/ |
|||
|
|||
func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer) error { |
|||
// 1. write to the volume server
|
|||
// 2. find the topic metadata owning filer
|
|||
// 3. write to the filer
|
|||
|
|||
var localTopicPartition *topic.LocalPartition |
|||
for { |
|||
req, err := stream.Recv() |
|||
if err != nil { |
|||
return err |
|||
} |
|||
|
|||
// Process the received message
|
|||
sequence := req.GetSequence() |
|||
response := &mq_pb.PublishResponse{ |
|||
AckSequence: sequence, |
|||
} |
|||
if dataMessage := req.GetData(); dataMessage != nil { |
|||
if localTopicPartition == nil { |
|||
response.Error = "topic partition not initialized" |
|||
glog.Errorf("topic partition not found") |
|||
} else { |
|||
localTopicPartition.Publish(dataMessage) |
|||
} |
|||
} else if initMessage := req.GetInit(); initMessage != nil { |
|||
localTopicPartition = broker.localTopicManager.GetTopicPartition( |
|||
topic.NewTopic(topic.Namespace(initMessage.Segment.Namespace), initMessage.Segment.Topic), |
|||
topic.FromPbPartition(initMessage.Segment.Partition), |
|||
) |
|||
if localTopicPartition == nil { |
|||
response.Error = fmt.Sprintf("topic partition %v not found", initMessage.Segment) |
|||
glog.Errorf("topic partition %v not found", initMessage.Segment) |
|||
} |
|||
} |
|||
if err := stream.Send(response); err != nil { |
|||
glog.Errorf("Error sending setup response: %v", err) |
|||
} |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment
|
|||
func (broker *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) { |
|||
ret := &mq_pb.AssignTopicPartitionsResponse{} |
|||
self := pb.ServerAddress(fmt.Sprintf("%s:%d", broker.option.Ip, broker.option.Port)) |
|||
|
|||
for _, partition := range request.TopicPartitionsAssignment.BrokerPartitions { |
|||
localPartiton := topic.FromPbBrokerPartitionsAssignment(self, partition) |
|||
broker.localTopicManager.AddTopicPartition( |
|||
topic.FromPbTopic(request.Topic), |
|||
localPartiton) |
|||
if request.IsLeader { |
|||
for _, follower := range localPartiton.FollowerBrokers { |
|||
err := pb.WithBrokerClient(false, follower, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { |
|||
_, err := client.AssignTopicPartitions(context.Background(), request) |
|||
return err |
|||
}) |
|||
if err != nil { |
|||
return ret, err |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
return ret, nil |
|||
} |
@ -0,0 +1,54 @@ |
|||
package topic |
|||
|
|||
import ( |
|||
cmap "github.com/orcaman/concurrent-map/v2" |
|||
) |
|||
|
|||
// LocalTopicManager manages topics on local broker
|
|||
type LocalTopicManager struct { |
|||
topics cmap.ConcurrentMap[string, *LocalTopic] |
|||
} |
|||
|
|||
// NewLocalTopicManager creates a new LocalTopicManager
|
|||
func NewLocalTopicManager() *LocalTopicManager { |
|||
return &LocalTopicManager{ |
|||
topics: cmap.New[*LocalTopic](), |
|||
} |
|||
} |
|||
|
|||
// AddTopic adds a topic to the local topic manager
|
|||
func (manager *LocalTopicManager) AddTopicPartition(topic Topic, localPartition *LocalPartition) { |
|||
localTopic, ok := manager.topics.Get(topic.String()) |
|||
if !ok { |
|||
localTopic = &LocalTopic{ |
|||
Topic: topic, |
|||
Partitions: make([]*LocalPartition, 0), |
|||
} |
|||
} |
|||
if localTopic.findPartition(localPartition.Partition) != nil { |
|||
return |
|||
} |
|||
localTopic.Partitions = append(localTopic.Partitions, localPartition) |
|||
} |
|||
|
|||
// GetTopic gets a topic from the local topic manager
|
|||
func (manager *LocalTopicManager) GetTopicPartition(topic Topic, partition Partition) *LocalPartition { |
|||
localTopic, ok := manager.topics.Get(topic.String()) |
|||
if !ok { |
|||
return nil |
|||
} |
|||
return localTopic.findPartition(partition) |
|||
} |
|||
|
|||
// RemoveTopic removes a topic from the local topic manager
|
|||
func (manager *LocalTopicManager) RemoveTopic(topic Topic) { |
|||
manager.topics.Remove(topic.String()) |
|||
} |
|||
|
|||
func (manager *LocalTopicManager) RemoveTopicPartition(topic Topic, partition Partition) (removed bool) { |
|||
localTopic, ok := manager.topics.Get(topic.String()) |
|||
if !ok { |
|||
return false |
|||
} |
|||
return localTopic.removePartition(partition) |
|||
} |
@ -0,0 +1,40 @@ |
|||
package topic |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer" |
|||
"time" |
|||
) |
|||
|
|||
type LocalPartition struct { |
|||
Partition |
|||
isLeader bool |
|||
FollowerBrokers []pb.ServerAddress |
|||
logBuffer *log_buffer.LogBuffer |
|||
} |
|||
|
|||
func (p LocalPartition) Publish(message *mq_pb.PublishRequest_DataMessage) { |
|||
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano()) |
|||
} |
|||
|
|||
func FromPbBrokerPartitionsAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionsAssignment) *LocalPartition { |
|||
isLeaer := assignment.LeaderBroker == string(self) |
|||
localPartition := &LocalPartition{ |
|||
Partition: Partition{ |
|||
RangeStart: assignment.PartitionStart, |
|||
RangeStop: assignment.PartitionStop, |
|||
RingSize: PartitionCount, |
|||
}, |
|||
isLeader: isLeaer, |
|||
} |
|||
if !isLeaer { |
|||
return localPartition |
|||
} |
|||
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers)) |
|||
for i, follower := range assignment.FollowerBrokers { |
|||
followers[i] = pb.ServerAddress(follower) |
|||
} |
|||
localPartition.FollowerBrokers = followers |
|||
return localPartition |
|||
} |
@ -0,0 +1,29 @@ |
|||
package topic |
|||
|
|||
type LocalTopic struct { |
|||
Topic |
|||
Partitions []*LocalPartition |
|||
} |
|||
|
|||
func (localTopic *LocalTopic) findPartition(partition Partition) *LocalPartition { |
|||
for _, localPartition := range localTopic.Partitions { |
|||
if localPartition.Partition.Equals(partition) { |
|||
return localPartition |
|||
} |
|||
} |
|||
return nil |
|||
} |
|||
func (localTopic *LocalTopic) removePartition(partition Partition) bool { |
|||
foundPartitionIndex := -1 |
|||
for i, localPartition := range localTopic.Partitions { |
|||
if localPartition.Partition.Equals(partition) { |
|||
foundPartitionIndex = i |
|||
break |
|||
} |
|||
} |
|||
if foundPartitionIndex == -1 { |
|||
return false |
|||
} |
|||
localTopic.Partitions = append(localTopic.Partitions[:foundPartitionIndex], localTopic.Partitions[foundPartitionIndex+1:]...) |
|||
return true |
|||
} |
@ -0,0 +1,32 @@ |
|||
package topic |
|||
|
|||
import "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
|||
|
|||
const PartitionCount = 4096 |
|||
|
|||
type Partition struct { |
|||
RangeStart int32 |
|||
RangeStop int32 // exclusive
|
|||
RingSize int32 |
|||
} |
|||
|
|||
func (partition Partition) Equals(other Partition) bool { |
|||
if partition.RangeStart != other.RangeStart { |
|||
return false |
|||
} |
|||
if partition.RangeStop != other.RangeStop { |
|||
return false |
|||
} |
|||
if partition.RingSize != other.RingSize { |
|||
return false |
|||
} |
|||
return true |
|||
} |
|||
|
|||
func FromPbPartition(partition *mq_pb.Partition) Partition { |
|||
return Partition{ |
|||
RangeStart: partition.RangeStart, |
|||
RangeStop: partition.RangeStop, |
|||
RingSize: partition.RingSize, |
|||
} |
|||
} |
@ -0,0 +1,81 @@ |
|||
package topic_allocation |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
|||
"modernc.org/mathutil" |
|||
) |
|||
|
|||
const ( |
|||
DefaultBrokerCount = 4 |
|||
) |
|||
|
|||
// AllocateBrokersForTopicPartitions allocate brokers for a topic's all partitions
|
|||
func AllocateBrokersForTopicPartitions(t topic.Topic, prevAssignment *mq_pb.TopicPartitionsAssignment, candidateBrokers []pb.ServerAddress) (assignment *mq_pb.TopicPartitionsAssignment, err error) { |
|||
// create a previous assignment if not exists
|
|||
if prevAssignment == nil || len(prevAssignment.BrokerPartitions) == 0 { |
|||
prevAssignment = &mq_pb.TopicPartitionsAssignment{ |
|||
PartitionCount: topic.PartitionCount, |
|||
} |
|||
partitionCountForEachBroker := topic.PartitionCount / DefaultBrokerCount |
|||
for i := 0; i < DefaultBrokerCount; i++ { |
|||
prevAssignment.BrokerPartitions = append(prevAssignment.BrokerPartitions, &mq_pb.BrokerPartitionsAssignment{ |
|||
PartitionStart: int32(i * partitionCountForEachBroker), |
|||
PartitionStop: mathutil.MaxInt32(int32((i+1)*partitionCountForEachBroker), topic.PartitionCount), |
|||
}) |
|||
} |
|||
} |
|||
|
|||
// create a new assignment
|
|||
assignment = &mq_pb.TopicPartitionsAssignment{ |
|||
PartitionCount: prevAssignment.PartitionCount, |
|||
} |
|||
|
|||
// allocate partitions for each partition range
|
|||
for _, brokerPartition := range prevAssignment.BrokerPartitions { |
|||
// allocate partitions for each partition range
|
|||
leader, followers, err := allocateBrokersForOneTopicPartition(t, brokerPartition, candidateBrokers) |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
followerBrokers := make([]string, len(followers)) |
|||
for i, follower := range followers { |
|||
followerBrokers[i] = string(follower) |
|||
} |
|||
|
|||
assignment.BrokerPartitions = append(assignment.BrokerPartitions, &mq_pb.BrokerPartitionsAssignment{ |
|||
PartitionStart: brokerPartition.PartitionStart, |
|||
PartitionStop: brokerPartition.PartitionStop, |
|||
LeaderBroker: string(leader), |
|||
FollowerBrokers: followerBrokers, |
|||
}) |
|||
} |
|||
|
|||
return |
|||
} |
|||
|
|||
func allocateBrokersForOneTopicPartition(t topic.Topic, brokerPartition *mq_pb.BrokerPartitionsAssignment, candidateBrokers []pb.ServerAddress) (leader pb.ServerAddress, followers []pb.ServerAddress, err error) { |
|||
// allocate leader
|
|||
leader, err = allocateLeaderForOneTopicPartition(t, brokerPartition, candidateBrokers) |
|||
if err != nil { |
|||
return |
|||
} |
|||
|
|||
// allocate followers
|
|||
followers, err = allocateFollowersForOneTopicPartition(t, brokerPartition, candidateBrokers) |
|||
if err != nil { |
|||
return |
|||
} |
|||
|
|||
return |
|||
} |
|||
|
|||
func allocateFollowersForOneTopicPartition(t topic.Topic, partition *mq_pb.BrokerPartitionsAssignment, brokers []pb.ServerAddress) (followers []pb.ServerAddress, err error) { |
|||
return |
|||
} |
|||
|
|||
func allocateLeaderForOneTopicPartition(t topic.Topic, partition *mq_pb.BrokerPartitionsAssignment, brokers []pb.ServerAddress) (leader pb.ServerAddress, err error) { |
|||
return |
|||
} |
1535
weed/pb/mq_pb/mq.pb.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
Write
Preview
Loading…
Cancel
Save
Reference in new issue