Browse Source

refactor

pull/5637/head
chrislu 1 year ago
parent
commit
cbf750a31f
  1. 6
      weed/mq/client/cmd/weed_pub/publisher.go
  2. 5
      weed/mq/client/pub_client/connect.go
  3. 8
      weed/mq/client/pub_client/publisher.go
  4. 31
      weed/mq/client/pub_client/scheduler.go

6
weed/mq/client/cmd/weed_pub/publisher.go

@ -4,6 +4,7 @@ import (
"flag" "flag"
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"log" "log"
"strings" "strings"
"sync" "sync"
@ -16,7 +17,7 @@ var (
partitionCount = flag.Int("p", 6, "partition count") partitionCount = flag.Int("p", 6, "partition count")
namespace = flag.String("ns", "test", "namespace") namespace = flag.String("ns", "test", "namespace")
topic = flag.String("topic", "test", "topic")
t = flag.String("t", "test", "t")
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
) )
@ -39,10 +40,11 @@ func doPublish(publisher *pub_client.TopicPublisher, id int) {
func main() { func main() {
flag.Parse() flag.Parse()
config := &pub_client.PublisherConfiguration{ config := &pub_client.PublisherConfiguration{
Topic: topic.NewTopic(*namespace, *t),
CreateTopic: true, CreateTopic: true,
CreateTopicPartitionCount: int32(*partitionCount), CreateTopicPartitionCount: int32(*partitionCount),
} }
publisher := pub_client.NewTopicPublisher(*namespace, *topic, config)
publisher := pub_client.NewTopicPublisher(config)
wg := sync.WaitGroup{} wg := sync.WaitGroup{}
wg.Add(1) wg.Add(1)
go func() { go func() {

5
weed/mq/client/pub_client/connect.go

@ -32,10 +32,7 @@ func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress str
if err = publishClient.Send(&mq_pb.PublishMessageRequest{ if err = publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Init{ Message: &mq_pb.PublishMessageRequest_Init{
Init: &mq_pb.PublishMessageRequest_InitMessage{ Init: &mq_pb.PublishMessageRequest_InitMessage{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Topic: p.config.Topic.ToPbTopic(),
Partition: &mq_pb.Partition{ Partition: &mq_pb.Partition{
RingSize: partition.RingSize, RingSize: partition.RingSize,
RangeStart: partition.RangeStart, RangeStart: partition.RangeStart,

8
weed/mq/client/pub_client/publisher.go

@ -3,6 +3,7 @@ package pub_client
import ( import (
"github.com/rdleal/intervalst/interval" "github.com/rdleal/intervalst/interval"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue" "github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
"google.golang.org/grpc" "google.golang.org/grpc"
@ -12,6 +13,7 @@ import (
) )
type PublisherConfiguration struct { type PublisherConfiguration struct {
Topic topic.Topic
CreateTopic bool CreateTopic bool
CreateTopicPartitionCount int32 CreateTopicPartitionCount int32
} }
@ -22,8 +24,6 @@ type PublishClient struct {
Err error Err error
} }
type TopicPublisher struct { type TopicPublisher struct {
namespace string
topic string
partition2Buffer *interval.SearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage], int32] partition2Buffer *interval.SearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage], int32]
grpcDialOption grpc.DialOption grpcDialOption grpc.DialOption
sync.Mutex // protects grpc sync.Mutex // protects grpc
@ -31,10 +31,8 @@ type TopicPublisher struct {
jobs []*EachPartitionPublishJob jobs []*EachPartitionPublishJob
} }
func NewTopicPublisher(namespace, topic string, config *PublisherConfiguration) *TopicPublisher {
func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher {
return &TopicPublisher{ return &TopicPublisher{
namespace: namespace,
topic: topic,
partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int { partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int {
return int(a - b) return int(a - b)
}), }),

31
weed/mq/client/pub_client/scheduler.go

@ -31,15 +31,15 @@ type EachPartitionPublishJob struct {
func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string, wg *sync.WaitGroup) error { func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string, wg *sync.WaitGroup) error {
if err := p.doEnsureConfigureTopic(bootstrapBrokers); err != nil { if err := p.doEnsureConfigureTopic(bootstrapBrokers); err != nil {
return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err)
return fmt.Errorf("configure topic %s: %v", p.config.Topic, err)
} }
log.Printf("start scheduler thread for topic %s/%s", p.namespace, p.topic)
log.Printf("start scheduler thread for topic %s", p.config.Topic)
generation := 0 generation := 0
var errChan chan EachPartitionError var errChan chan EachPartitionError
for { for {
glog.V(0).Infof("lookup partitions gen %d topic %s/%s", generation+1, p.namespace, p.topic)
glog.V(0).Infof("lookup partitions gen %d topic %s", generation+1, p.config.Topic)
if assignments, err := p.doLookupTopicPartitions(bootstrapBrokers); err == nil { if assignments, err := p.doLookupTopicPartitions(bootstrapBrokers); err == nil {
generation++ generation++
glog.V(0).Infof("start generation %d with %d assignments", generation, len(assignments)) glog.V(0).Infof("start generation %d with %d assignments", generation, len(assignments))
@ -48,7 +48,7 @@ func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string, wg *syn
} }
p.onEachAssignments(generation, assignments, errChan) p.onEachAssignments(generation, assignments, errChan)
} else { } else {
glog.Errorf("lookup topic %s/%s: %v", p.namespace, p.topic, err)
glog.Errorf("lookup topic %s: %v", p.config.Topic, err)
time.Sleep(5 * time.Second) time.Sleep(5 * time.Second)
continue continue
} }
@ -61,7 +61,7 @@ func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string, wg *syn
for { for {
select { select {
case eachErr := <-errChan: case eachErr := <-errChan:
glog.Errorf("gen %d publish to topic %s/%s partition %v: %v", eachErr.generation, p.namespace, p.topic, eachErr.Partition, eachErr.Err)
glog.Errorf("gen %d publish to topic %s partition %v: %v", eachErr.generation, p.config.Topic, eachErr.Partition, eachErr.Err)
if eachErr.generation < generation { if eachErr.generation < generation {
continue continue
} }
@ -140,10 +140,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro
if err = publishClient.Send(&mq_pb.PublishMessageRequest{ if err = publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Init{ Message: &mq_pb.PublishMessageRequest_Init{
Init: &mq_pb.PublishMessageRequest_InitMessage{ Init: &mq_pb.PublishMessageRequest_InitMessage{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Topic: p.config.Topic.ToPbTopic(),
Partition: job.Partition, Partition: job.Partition,
AckInterval: 128, AckInterval: 128,
}, },
@ -197,10 +194,7 @@ func (p *TopicPublisher) doEnsureConfigureTopic(bootstrapBrokers []string) (err
p.grpcDialOption, p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error { func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{ _, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Topic: p.config.Topic.ToPbTopic(),
PartitionCount: p.config.CreateTopicPartitionCount, PartitionCount: p.config.CreateTopicPartitionCount,
}) })
return err return err
@ -213,7 +207,7 @@ func (p *TopicPublisher) doEnsureConfigureTopic(bootstrapBrokers []string) (err
} }
if lastErr != nil { if lastErr != nil {
return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err)
return fmt.Errorf("configure topic %s: %v", p.config.Topic, err)
} }
return nil return nil
} }
@ -230,12 +224,9 @@ func (p *TopicPublisher) doLookupTopicPartitions(bootstrapBrokers []string) (ass
func(client mq_pb.SeaweedMessagingClient) error { func(client mq_pb.SeaweedMessagingClient) error {
lookupResp, err := client.LookupTopicBrokers(context.Background(), lookupResp, err := client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{ &mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Topic: p.config.Topic.ToPbTopic(),
}) })
glog.V(0).Infof("lookup topic %s/%s: %v", p.namespace, p.topic, lookupResp)
glog.V(0).Infof("lookup topic %s: %v", p.config.Topic, lookupResp)
if err != nil { if err != nil {
return err return err
@ -256,6 +247,6 @@ func (p *TopicPublisher) doLookupTopicPartitions(bootstrapBrokers []string) (ass
} }
} }
return nil, fmt.Errorf("lookup topic %s/%s: %v", p.namespace, p.topic, lastErr)
return nil, fmt.Errorf("lookup topic %s: %v", p.config.Topic, lastErr)
} }
Loading…
Cancel
Save