mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-10-15 20:06:19 +08:00
subscriber keep connected to the balancer
This commit is contained in:
@@ -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))
|
||||
|
91
weed/mq/client/sub_client/connect_to_sub_coordinator.go
Normal file
91
weed/mq/client/sub_client/connect_to_sub_coordinator.go
Normal 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)
|
||||
}
|
@@ -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,
|
||||
|
@@ -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++
|
||||
|
@@ -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(),
|
||||
|
Reference in New Issue
Block a user