mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
publish, benchmark
This commit is contained in:
parent
cb470d44df
commit
9e4f985698
|
@ -87,6 +87,7 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS
|
||||||
localTopicPartition = topic.NewLocalPartition(t, p, true, nil)
|
localTopicPartition = topic.NewLocalPartition(t, p, true, nil)
|
||||||
broker.localTopicManager.AddTopicPartition(t, localTopicPartition)
|
broker.localTopicManager.AddTopicPartition(t, localTopicPartition)
|
||||||
}
|
}
|
||||||
|
stream.Send(response)
|
||||||
} else {
|
} else {
|
||||||
response.Error = fmt.Sprintf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
|
response.Error = fmt.Sprintf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
|
||||||
glog.Errorf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
|
glog.Errorf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
|
||||||
|
@ -106,7 +107,6 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS
|
||||||
AckSequence: sequence,
|
AckSequence: sequence,
|
||||||
}
|
}
|
||||||
if dataMessage := req.GetData(); dataMessage != nil {
|
if dataMessage := req.GetData(); dataMessage != nil {
|
||||||
print("+")
|
|
||||||
localTopicPartition.Publish(dataMessage)
|
localTopicPartition.Publish(dataMessage)
|
||||||
}
|
}
|
||||||
if err := stream.Send(response); err != nil {
|
if err := stream.Send(response); err != nil {
|
||||||
|
@ -114,6 +114,8 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
glog.Infof("publish stream closed")
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,12 +1,34 @@
|
||||||
package main
|
package main
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"flag"
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
|
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
|
||||||
|
"log"
|
||||||
|
"sync"
|
||||||
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
func main() {
|
var (
|
||||||
|
messageCount = flag.Int("n", 1000, "message count")
|
||||||
|
concurrency = flag.Int("c", 4, "concurrency count")
|
||||||
|
)
|
||||||
|
|
||||||
|
func doPublish(publisher *pub_client.TopicPublisher, id int) {
|
||||||
|
startTime := time.Now()
|
||||||
|
for i := 0; i < *messageCount / *concurrency; i++ {
|
||||||
|
// Simulate publishing a message
|
||||||
|
key := []byte(fmt.Sprintf("key-%d-%d", id, i))
|
||||||
|
value := []byte(fmt.Sprintf("value-%d-%d", id, i))
|
||||||
|
publisher.Publish(key, value) // Call your publisher function here
|
||||||
|
// println("Published", string(key), string(value))
|
||||||
|
}
|
||||||
|
elapsed := time.Since(startTime)
|
||||||
|
log.Printf("Publisher %d finished in %s", id, elapsed)
|
||||||
|
}
|
||||||
|
|
||||||
|
func main() {
|
||||||
|
flag.Parse()
|
||||||
publisher := pub_client.NewTopicPublisher(
|
publisher := pub_client.NewTopicPublisher(
|
||||||
"test", "test")
|
"test", "test")
|
||||||
if err := publisher.Connect("localhost:17777"); err != nil {
|
if err := publisher.Connect("localhost:17777"); err != nil {
|
||||||
|
@ -14,16 +36,22 @@ func main() {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
for i := 0; i < 10; i++ {
|
startTime := time.Now()
|
||||||
if dataErr := publisher.Publish(
|
|
||||||
[]byte(fmt.Sprintf("key-%d", i)),
|
// Start multiple publishers
|
||||||
[]byte(fmt.Sprintf("value-%d", i)),
|
var wg sync.WaitGroup
|
||||||
); dataErr != nil {
|
for i := 0; i < *concurrency; i++ {
|
||||||
fmt.Println(dataErr)
|
wg.Add(1)
|
||||||
return
|
go func(id int) {
|
||||||
}
|
defer wg.Done()
|
||||||
|
doPublish(publisher, id)
|
||||||
|
}(i)
|
||||||
}
|
}
|
||||||
|
|
||||||
fmt.Println("done publishing")
|
// Wait for all publishers to finish
|
||||||
|
wg.Wait()
|
||||||
|
elapsed := time.Since(startTime)
|
||||||
|
|
||||||
|
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,43 +24,21 @@ func (p *TopicPublisher) doLookup(brokerAddress string) error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments {
|
for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments {
|
||||||
// partition => broker
|
// partition => publishClient
|
||||||
|
publishClient, redirectTo, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
for redirectTo != "" {
|
||||||
|
publishClient, redirectTo, err = p.doConnect(brokerPartitionAssignment.Partition, redirectTo)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
p.partition2Broker.Insert(
|
p.partition2Broker.Insert(
|
||||||
brokerPartitionAssignment.Partition.RangeStart,
|
brokerPartitionAssignment.Partition.RangeStart,
|
||||||
brokerPartitionAssignment.Partition.RangeStop,
|
brokerPartitionAssignment.Partition.RangeStop,
|
||||||
brokerPartitionAssignment.LeaderBroker)
|
publishClient)
|
||||||
|
|
||||||
// broker => publish client
|
|
||||||
// send init message
|
|
||||||
// save the publishing client
|
|
||||||
brokerAddress := brokerPartitionAssignment.LeaderBroker
|
|
||||||
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("dial broker %s: %v", brokerAddress, err)
|
|
||||||
}
|
|
||||||
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
|
|
||||||
publishClient, err := brokerClient.Publish(context.Background())
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("create publish client: %v", err)
|
|
||||||
}
|
|
||||||
p.broker2PublishClient.Set(brokerAddress, publishClient)
|
|
||||||
if err = publishClient.Send(&mq_pb.PublishRequest{
|
|
||||||
Message: &mq_pb.PublishRequest_Init{
|
|
||||||
Init: &mq_pb.PublishRequest_InitMessage{
|
|
||||||
Topic: &mq_pb.Topic{
|
|
||||||
Namespace: p.namespace,
|
|
||||||
Name: p.topic,
|
|
||||||
},
|
|
||||||
Partition: &mq_pb.Partition{
|
|
||||||
RingSize: brokerPartitionAssignment.Partition.RingSize,
|
|
||||||
RangeStart: brokerPartitionAssignment.Partition.RangeStart,
|
|
||||||
RangeStop: brokerPartitionAssignment.Partition.RangeStop,
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}); err != nil {
|
|
||||||
return fmt.Errorf("send init message: %v", err)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
})
|
})
|
||||||
|
@ -70,3 +48,62 @@ func (p *TopicPublisher) doLookup(brokerAddress string) error {
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// broker => publish client
|
||||||
|
// send init message
|
||||||
|
// save the publishing client
|
||||||
|
func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress string) (publishClient *PublishClient, redirectTo string, err error) {
|
||||||
|
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption)
|
||||||
|
if err != nil {
|
||||||
|
return publishClient, redirectTo, fmt.Errorf("dial broker %s: %v", brokerAddress, err)
|
||||||
|
}
|
||||||
|
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
|
||||||
|
stream, err := brokerClient.Publish(context.Background())
|
||||||
|
if err != nil {
|
||||||
|
return publishClient, redirectTo, fmt.Errorf("create publish client: %v", err)
|
||||||
|
}
|
||||||
|
publishClient = &PublishClient{
|
||||||
|
SeaweedMessaging_PublishClient: stream,
|
||||||
|
Broker: brokerAddress,
|
||||||
|
}
|
||||||
|
if err = publishClient.Send(&mq_pb.PublishRequest{
|
||||||
|
Message: &mq_pb.PublishRequest_Init{
|
||||||
|
Init: &mq_pb.PublishRequest_InitMessage{
|
||||||
|
Topic: &mq_pb.Topic{
|
||||||
|
Namespace: p.namespace,
|
||||||
|
Name: p.topic,
|
||||||
|
},
|
||||||
|
Partition: &mq_pb.Partition{
|
||||||
|
RingSize: partition.RingSize,
|
||||||
|
RangeStart: partition.RangeStart,
|
||||||
|
RangeStop: partition.RangeStop,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}); err != nil {
|
||||||
|
return publishClient, redirectTo, fmt.Errorf("send init message: %v", err)
|
||||||
|
}
|
||||||
|
resp, err := stream.Recv()
|
||||||
|
if err != nil {
|
||||||
|
return publishClient, redirectTo, fmt.Errorf("recv init response: %v", err)
|
||||||
|
}
|
||||||
|
if resp.Error != "" {
|
||||||
|
return publishClient, redirectTo, fmt.Errorf("init response error: %v", resp.Error)
|
||||||
|
}
|
||||||
|
if resp.RedirectToBroker != "" {
|
||||||
|
redirectTo = resp.RedirectToBroker
|
||||||
|
return publishClient, redirectTo, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
for {
|
||||||
|
_, err := publishClient.Recv()
|
||||||
|
if err != nil {
|
||||||
|
publishClient.Err = err
|
||||||
|
fmt.Printf("publish to %s error: %v\n", publishClient.Broker, err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
return publishClient, redirectTo, nil
|
||||||
|
}
|
||||||
|
|
|
@ -12,14 +12,21 @@ func (p *TopicPublisher) Publish(key, value []byte) error {
|
||||||
if hashKey < 0 {
|
if hashKey < 0 {
|
||||||
hashKey = -hashKey
|
hashKey = -hashKey
|
||||||
}
|
}
|
||||||
brokerAddress, found := p.partition2Broker.Floor(hashKey, hashKey)
|
publishClient, found := p.partition2Broker.Floor(hashKey, hashKey)
|
||||||
if !found {
|
if !found {
|
||||||
return fmt.Errorf("no broker found for key %d", hashKey)
|
return fmt.Errorf("no broker found for key %d", hashKey)
|
||||||
}
|
}
|
||||||
publishClient, found := p.broker2PublishClient.Get(brokerAddress)
|
p.Lock()
|
||||||
if !found {
|
defer p.Unlock()
|
||||||
return fmt.Errorf("no publish client found for broker %s", brokerAddress)
|
// 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.PublishRequest{
|
if err := publishClient.Send(&mq_pb.PublishRequest{
|
||||||
Message: &mq_pb.PublishRequest_Data{
|
Message: &mq_pb.PublishRequest_Data{
|
||||||
Data: &mq_pb.DataMessage{
|
Data: &mq_pb.DataMessage{
|
||||||
|
|
|
@ -1,31 +1,36 @@
|
||||||
package pub_client
|
package pub_client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
|
||||||
"github.com/rdleal/intervalst/interval"
|
"github.com/rdleal/intervalst/interval"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/credentials/insecure"
|
"google.golang.org/grpc/credentials/insecure"
|
||||||
|
"sync"
|
||||||
)
|
)
|
||||||
|
|
||||||
type PublisherConfiguration struct {
|
type PublisherConfiguration struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type PublishClient struct {
|
||||||
|
mq_pb.SeaweedMessaging_PublishClient
|
||||||
|
Broker string
|
||||||
|
Err error
|
||||||
|
}
|
||||||
type TopicPublisher struct {
|
type TopicPublisher struct {
|
||||||
namespace string
|
namespace string
|
||||||
topic string
|
topic string
|
||||||
partition2Broker *interval.SearchTree[string, int32]
|
partition2Broker *interval.SearchTree[*PublishClient, int32]
|
||||||
broker2PublishClient cmap.ConcurrentMap[string, mq_pb.SeaweedMessaging_PublishClient]
|
|
||||||
grpcDialOption grpc.DialOption
|
grpcDialOption grpc.DialOption
|
||||||
|
sync.Mutex // protects grpc
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewTopicPublisher(namespace, topic string) *TopicPublisher {
|
func NewTopicPublisher(namespace, topic string) *TopicPublisher {
|
||||||
return &TopicPublisher{
|
return &TopicPublisher{
|
||||||
namespace: namespace,
|
namespace: namespace,
|
||||||
topic: topic,
|
topic: topic,
|
||||||
partition2Broker: interval.NewSearchTree[string](func(a, b int32) int {
|
partition2Broker: interval.NewSearchTree[*PublishClient](func(a, b int32) int {
|
||||||
return int(a - b)
|
return int(a - b)
|
||||||
}),
|
}),
|
||||||
broker2PublishClient: cmap.New[mq_pb.SeaweedMessaging_PublishClient](),
|
|
||||||
grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue