subscriber keep connected to the balancer

This commit is contained in:
chrislu
2023-12-28 11:56:37 -08:00
parent bebbc9fe44
commit c950a40aad
9 changed files with 375 additions and 246 deletions

View File

@@ -23,14 +23,14 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
// process init message
initMessage := req.GetInit()
if initMessage != nil {
cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
} else {
return status.Errorf(codes.InvalidArgument, "subscriber init message is empty")
}
defer func() {
b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}()
ctx := stream.Context()
@@ -40,7 +40,7 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
for {
_, err := stream.Recv()
if err != nil {
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}
select {
@@ -66,11 +66,11 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
// Client disconnected
return err
}
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
return err
case message := <-cgi.ResponseChan:
if err := stream.Send(message); err != nil {
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}
}
}

View File

@@ -20,10 +20,10 @@ func main() {
flag.Parse()
subscriberConfig := &sub_client.SubscriberConfiguration{
ClientId: "testSubscriber",
GroupId: "test",
GroupInstanceId: "test",
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
ClientId: "testSubscriber",
ConsumerGroup: "test",
ConsumerGroupInstanceId: "test",
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
}
contentConfig := &sub_client.ContentConfiguration{
@@ -33,8 +33,12 @@ func main() {
StartTime: time.Now(),
}
processorConfig := sub_client.ProcessorConfiguration{
ConcurrentPartitionLimit: 1,
}
brokers := strings.Split(*seedBrokers, ",")
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig)
subscriber.SetEachMessageFunc(func(key, value []byte) bool {
println(string(key), "=>", string(value))

View File

@@ -0,0 +1,91 @@
package sub_client
import (
"context"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"sync"
"time"
)
func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
for {
for _, broker := range sub.bootstrapBrokers {
// TODO find the balancer
// connect to the balancer
pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stream, err := client.SubscriberToSubCoordinator(ctx)
if err != nil {
glog.V(1).Infof("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
return err
}
// Maybe later: subscribe to multiple topics instead of just one
if err := stream.Send(&mq_pb.SubscriberToSubCoordinatorRequest{
Message: &mq_pb.SubscriberToSubCoordinatorRequest_Init{
Init: &mq_pb.SubscriberToSubCoordinatorRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerGroupInstanceId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: &mq_pb.Topic{
Namespace: sub.ContentConfig.Namespace,
Name: sub.ContentConfig.Topic,
},
},
},
}); err != nil {
glog.V(1).Infof("subscriber %s/%s send init: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
return err
}
// keep receiving messages from the sub coordinator
for {
resp, err := stream.Recv()
if err != nil {
glog.V(1).Infof("subscriber %s/%s receive: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
return err
}
assignment := resp.GetAssignment()
if assignment != nil {
glog.V(0).Infof("subscriber %s/%s receive assignment: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, assignment)
}
sub.onEachAssignment(assignment)
}
return nil
})
}
print("z")
time.Sleep(3 * time.Second)
}
}
func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCoordinatorResponse_Assignment) {
if assignment == nil {
return
}
// process each partition, with a concurrency limit
var wg sync.WaitGroup
semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit)
for _, partition := range assignment.AssignedPartitions {
wg.Add(1)
semaphore <- struct{}{}
go func(partition *mq_pb.Partition) {
defer wg.Done()
defer func() { <-semaphore }()
glog.V(0).Infof("subscriber %s/%s/%s assigned partition %d", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition)
sub.onEachPartition(partition)
}(partition.Partition)
}
wg.Wait()
}
func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition) {
glog.V(0).Infof("subscriber %s/%s/%s processing partition %d", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition)
}

View File

@@ -20,8 +20,8 @@ func (sub *TopicSubscriber) doProcess() error {
subscribeClient, err := brokerClient.Subscribe(context.Background(), &mq_pb.SubscribeRequest{
Message: &mq_pb.SubscribeRequest_Init{
Init: &mq_pb.SubscribeRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.GroupId,
ConsumerId: sub.SubscriberConfig.GroupInstanceId,
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: &mq_pb.Topic{
Namespace: sub.ContentConfig.Namespace,
Name: sub.ContentConfig.Topic,

View File

@@ -12,6 +12,11 @@ import (
// If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
func (sub *TopicSubscriber) Subscribe() error {
// loop forever
sub.doKeepConnectedToSubCoordinator()
index := -1
util.RetryUntil("subscribe", func() error {
index++

View File

@@ -7,10 +7,10 @@ import (
)
type SubscriberConfiguration struct {
ClientId string
GroupId string
GroupInstanceId string
GroupMinimumPeers int32
ClientId string
ConsumerGroup string
ConsumerGroupInstanceId string
GroupMinimumPeers int32
GroupMaximumPeers int32
BootstrapServers []string
GrpcDialOption grpc.DialOption
@@ -23,12 +23,17 @@ type ContentConfiguration struct {
StartTime time.Time
}
type ProcessorConfiguration struct {
ConcurrentPartitionLimit int // how many partitions to process concurrently
}
type OnEachMessageFunc func(key, value []byte) (shouldContinue bool)
type OnCompletionFunc func()
type TopicSubscriber struct {
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
ProcessorConfig *ProcessorConfiguration
brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
@@ -37,10 +42,11 @@ type TopicSubscriber struct {
alreadyProcessedTsNs int64
}
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, processor ProcessorConfiguration) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
ProcessorConfig: &processor,
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
alreadyProcessedTsNs: content.StartTime.UnixNano(),

View File

@@ -5,6 +5,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"time"
)
type ConsumerGroupInstance struct {
@@ -16,7 +17,8 @@ type ConsumerGroupInstance struct {
type ConsumerGroup struct {
// map a consumer group instance id to a consumer group instance
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance]
mapping *PartitionConsumerMapping
mapping *PartitionConsumerMapping
reBalanceTimer *time.Timer
}
func NewConsumerGroup() *ConsumerGroup {
@@ -33,10 +35,30 @@ func NewConsumerGroupInstance(instanceId string) *ConsumerGroupInstance {
}
}
func (cg *ConsumerGroup) OnAddConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
cg.onConsumerGroupInstanceChange()
}
func (cg *ConsumerGroup) OnRemoveConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
cg.onConsumerGroupInstanceChange()
}
func (cg *ConsumerGroup) onConsumerGroupInstanceChange(){
if cg.reBalanceTimer != nil {
cg.reBalanceTimer.Stop()
cg.reBalanceTimer = nil
}
cg.reBalanceTimer = time.AfterFunc(5*time.Second, func() {
cg.Rebalance()
cg.reBalanceTimer = nil
})
}
func (cg *ConsumerGroup) OnPartitionListChange() {
if cg.reBalanceTimer != nil {
cg.reBalanceTimer.Stop()
cg.reBalanceTimer = nil
}
cg.Rebalance()
}
func (cg *ConsumerGroup) Rebalance() {
println("rebalance...")
}