Browse Source

publish to input buffer

currently, the input buffer may not exist when start to publish
pull/5637/head
chrislu 11 months ago
parent
commit
841fafd0a8
  1. 1
      weed/mq/broker/broker_grpc_lookup.go
  2. 5
      weed/mq/broker/broker_topic_conf_read_write.go
  3. 14
      weed/mq/client/cmd/weed_pub/publisher.go
  4. 86
      weed/mq/client/pub_client/lookup.go
  5. 32
      weed/mq/client/pub_client/publish.go
  6. 25
      weed/mq/client/pub_client/publisher.go
  7. 74
      weed/mq/client/pub_client/scheduler.go

1
weed/mq/broker/broker_grpc_lookup.go

@ -32,7 +32,6 @@ func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq
ret.Topic = request.Topic ret.Topic = request.Topic
if conf, err = b.readTopicConfFromFiler(t); err != nil { if conf, err = b.readTopicConfFromFiler(t); err != nil {
glog.V(0).Infof("lookup topic %s conf: %v", request.Topic, err) glog.V(0).Infof("lookup topic %s conf: %v", request.Topic, err)
ret.BrokerPartitionAssignments = conf.BrokerPartitionAssignments
} else { } else {
err = b.ensureTopicActiveAssignments(t, conf) err = b.ensureTopicActiveAssignments(t, conf)
} }

5
weed/mq/broker/broker_topic_conf_read_write.go

@ -37,8 +37,11 @@ func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb.
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name) topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
if err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { if err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
data, err := filer.ReadInsideFiler(client, topicDir, "topic.conf") data, err := filer.ReadInsideFiler(client, topicDir, "topic.conf")
if err == filer_pb.ErrNotFound {
return err
}
if err != nil { if err != nil {
return fmt.Errorf("read topic %v partition %v conf: %v", t, err)
return fmt.Errorf("read topic.conf of %v: %v", t, err)
} }
// parse into filer conf object // parse into filer conf object
conf = &mq_pb.ConfigureTopicResponse{} conf = &mq_pb.ConfigureTopicResponse{}

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

@ -30,7 +30,7 @@ func doPublish(publisher *pub_client.TopicPublisher, id int) {
fmt.Println(err) fmt.Println(err)
break break
} }
// println("Published", string(key), string(value))
println("Published", string(key), string(value))
} }
elapsed := time.Since(startTime) elapsed := time.Since(startTime)
log.Printf("Publisher %d finished in %s", id, elapsed) log.Printf("Publisher %d finished in %s", id, elapsed)
@ -43,11 +43,13 @@ func main() {
CreateTopicPartitionCount: int32(*partitionCount), CreateTopicPartitionCount: int32(*partitionCount),
} }
publisher := pub_client.NewTopicPublisher(*namespace, *topic, config) publisher := pub_client.NewTopicPublisher(*namespace, *topic, config)
brokers := strings.Split(*seedBrokers, ",")
if err := publisher.Connect(brokers); err != nil {
fmt.Println(err)
return
}
go func() {
brokers := strings.Split(*seedBrokers, ",")
if err := publisher.StartSchedulerThread(brokers); err != nil {
fmt.Println(err)
return
}
}()
startTime := time.Now() startTime := time.Now()

86
weed/mq/client/pub_client/lookup.go

@ -1,86 +0,0 @@
package pub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
func (p *TopicPublisher) doLookupAndConnect(brokerAddress string) error {
if p.config.CreateTopic {
err := pb.WithBrokerGrpcClient(true,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
PartitionCount: p.config.CreateTopicPartitionCount,
})
return err
})
if err != nil {
return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err)
}
}
err := pb.WithBrokerGrpcClient(true,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
lookupResp, err := client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
})
glog.V(0).Infof("lookup1 topic %s/%s: %v", p.namespace, p.topic, lookupResp)
if p.config.CreateTopic && err != nil {
_, err = client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
PartitionCount: p.config.CreateTopicPartitionCount,
})
if err != nil {
return err
}
lookupResp, err = client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
})
glog.V(0).Infof("lookup2 topic %s/%s: %v", p.namespace, p.topic, lookupResp)
}
if err != nil {
return err
}
for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments {
glog.V(0).Infof("topic %s/%s partition %v leader %s followers %v", p.namespace, p.topic, brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker, brokerPartitionAssignment.FollowerBrokers)
// partition => publishClient
publishClient, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker)
if err != nil {
return err
}
p.partition2Broker.Insert(
brokerPartitionAssignment.Partition.RangeStart,
brokerPartitionAssignment.Partition.RangeStop,
publishClient)
}
return nil
})
if err != nil {
return fmt.Errorf("lookup topic %s/%s: %v", p.namespace, p.topic, err)
}
return nil
}

32
weed/mq/client/pub_client/publish.go

@ -7,35 +7,19 @@ import (
"github.com/seaweedfs/seaweedfs/weed/util" "github.com/seaweedfs/seaweedfs/weed/util"
) )
func (p *TopicPublisher) Publish(key, value []byte) error { func (p *TopicPublisher) Publish(key, value []byte) error {
hashKey := util.HashToInt32(key) % pub_balancer.MaxPartitionCount hashKey := util.HashToInt32(key) % pub_balancer.MaxPartitionCount
if hashKey < 0 { if hashKey < 0 {
hashKey = -hashKey hashKey = -hashKey
} }
publishClient, found := p.partition2Broker.Floor(hashKey+1, hashKey+1)
inputBuffer, found := p.partition2Buffer.Floor(hashKey+1, hashKey+1)
if !found { if !found {
return fmt.Errorf("no broker found for key %d", hashKey)
}
p.Lock()
defer p.Unlock()
// dead lock here
//google.golang.org/grpc/internal/transport.(*writeQuota).get(flowcontrol.go:59)
//google.golang.org/grpc/internal/transport.(*http2Client).Write(http2_client.go:1047)
//google.golang.org/grpc.(*csAttempt).sendMsg(stream.go:1040)
//google.golang.org/grpc.(*clientStream).SendMsg.func2(stream.go:892)
//google.golang.org/grpc.(*clientStream).withRetry(stream.go:752)
//google.golang.org/grpc.(*clientStream).SendMsg(stream.go:894)
//github.com/seaweedfs/seaweedfs/weed/pb/mq_pb.(*seaweedMessagingPublishClient).Send(mq_grpc.pb.go:141)
//github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client.(*TopicPublisher).Publish(publish.go:19)
if err := publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Data{
Data: &mq_pb.DataMessage{
Key: key,
Value: value,
},
},
}); err != nil {
return fmt.Errorf("send publish request: %v", err)
return fmt.Errorf("no input buffer found for key %d", hashKey)
} }
return nil
return inputBuffer.Enqueue(&mq_pb.DataMessage{
Key: key,
Value: value,
})
} }

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

@ -1,10 +1,10 @@
package pub_client package pub_client
import ( import (
"fmt"
"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/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/credentials/insecure"
"sync" "sync"
@ -25,6 +25,7 @@ type TopicPublisher struct {
namespace string namespace string
topic string topic string
partition2Broker *interval.SearchTree[*PublishClient, int32] partition2Broker *interval.SearchTree[*PublishClient, 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
config *PublisherConfiguration config *PublisherConfiguration
@ -38,25 +39,14 @@ func NewTopicPublisher(namespace, topic string, config *PublisherConfiguration)
partition2Broker: interval.NewSearchTree[*PublishClient](func(a, b int32) int { partition2Broker: interval.NewSearchTree[*PublishClient](func(a, b int32) int {
return int(a - b) return int(a - b)
}), }),
partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int {
return int(a - b)
}),
grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
config: config, config: config,
} }
} }
func (p *TopicPublisher) Connect(bootstrapBrokers []string) (err error) {
if len(bootstrapBrokers) == 0 {
return nil
}
for _, b := range bootstrapBrokers {
err = p.doLookupAndConnect(b)
if err == nil {
return nil
}
fmt.Printf("failed to connect to %s: %v\n\n", b, err)
}
return err
}
func (p *TopicPublisher) Shutdown() error { func (p *TopicPublisher) Shutdown() error {
if clients, found := p.partition2Broker.AllIntersections(0, pub_balancer.MaxPartitionCount); found { if clients, found := p.partition2Broker.AllIntersections(0, pub_balancer.MaxPartitionCount); found {
@ -64,6 +54,11 @@ func (p *TopicPublisher) Shutdown() error {
client.CloseSend() client.CloseSend()
} }
} }
if inputBuffers, found := p.partition2Buffer.AllIntersections(0, pub_balancer.MaxPartitionCount); found {
for _, inputBuffer := range inputBuffers {
inputBuffer.CloseInput()
}
}
time.Sleep(1100 * time.Millisecond) time.Sleep(1100 * time.Millisecond)
return nil return nil

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

@ -6,6 +6,10 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"log"
"sort" "sort"
"sync" "sync"
"time" "time"
@ -22,13 +26,16 @@ type EachPartitionPublishJob struct {
stopChan chan bool stopChan chan bool
wg sync.WaitGroup wg sync.WaitGroup
generation int generation int
inputQueue *buffered_queue.BufferedQueue[*mq_pb.DataMessage]
} }
func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string) error { func (p *TopicPublisher) StartSchedulerThread(bootstrapBrokers []string) error {
if err := p.doEnsureConfigureTopic(bootstrapBrokers); err != nil { if err := p.doEnsureConfigureTopic(bootstrapBrokers); err != nil {
return err
return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err)
} }
log.Printf("start scheduler thread for topic %s/%s", p.namespace, p.topic)
generation := 0 generation := 0
var errChan chan EachPartitionError var errChan chan EachPartitionError
for { for {
@ -92,6 +99,7 @@ func (p *TopicPublisher) onEachAssignments(generation int, assignments []*mq_pb.
BrokerPartitionAssignment: assignment, BrokerPartitionAssignment: assignment,
stopChan: make(chan bool, 1), stopChan: make(chan bool, 1),
generation: generation, generation: generation,
inputQueue: buffered_queue.NewBufferedQueue[*mq_pb.DataMessage](1024, true),
} }
job.wg.Add(1) job.wg.Add(1)
go func(job *EachPartitionPublishJob) { go func(job *EachPartitionPublishJob) {
@ -101,12 +109,76 @@ func (p *TopicPublisher) onEachAssignments(generation int, assignments []*mq_pb.
} }
}(job) }(job)
jobs = append(jobs, job) jobs = append(jobs, job)
// TODO assuming this is not re-configured so the partitions are fixed.
// better just re-use the existing job
p.partition2Buffer.Insert(assignment.Partition.RangeStart, assignment.Partition.RangeStop, job.inputQueue)
} }
p.jobs = jobs p.jobs = jobs
} }
func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) error { func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) error {
log.Printf("connecting to %v for topic partition %+v", job.LeaderBroker, job.Partition)
grpcConnection, err := pb.GrpcDial(context.Background(), job.LeaderBroker, true, p.grpcDialOption)
if err != nil {
return fmt.Errorf("dial broker %s: %v", job.LeaderBroker, err)
}
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
stream, err := brokerClient.PublishMessage(context.Background())
if err != nil {
return fmt.Errorf("create publish client: %v", err)
}
publishClient := &PublishClient{
SeaweedMessaging_PublishMessageClient: stream,
Broker: job.LeaderBroker,
}
if err = publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Init{
Init: &mq_pb.PublishMessageRequest_InitMessage{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Partition: job.Partition,
AckInterval: 128,
},
},
}); err != nil {
return fmt.Errorf("send init message: %v", err)
}
resp, err := stream.Recv()
if err != nil {
return fmt.Errorf("recv init response: %v", err)
}
if resp.Error != "" {
return fmt.Errorf("init response error: %v", resp.Error)
}
go func() {
for {
_, err := publishClient.Recv()
if err != nil {
e, ok := status.FromError(err)
if ok && e.Code() == codes.Unknown && e.Message() == "EOF" {
return
}
publishClient.Err = err
fmt.Printf("publish to %s error: %v\n", publishClient.Broker, err)
return
}
}
}()
for data, hasData := job.inputQueue.Dequeue(); hasData; data, hasData = job.inputQueue.Dequeue() {
if err := publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Data{
Data: data,
},
}); err != nil {
return fmt.Errorf("send publish data: %v", err)
}
}
return nil return nil
} }

Loading…
Cancel
Save