add broker connects to filer

This commit is contained in:
Chris Lu
2020-05-05 02:05:28 -07:00
parent 47234760f4
commit 1e3e4b3072
17 changed files with 761 additions and 327 deletions

View File

@@ -0,0 +1,56 @@
package msgclient
import (
"context"
"fmt"
"log"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/messaging/broker"
"github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/util"
)
type MessagingClient struct {
bootstrapBrokers []string
grpcConnections map[broker.TopicPartition]*grpc.ClientConn
grpcDialOption grpc.DialOption
}
func NewMessagingClient(bootstrapBrokers ...string) *MessagingClient {
return &MessagingClient{
bootstrapBrokers: bootstrapBrokers,
grpcConnections: make(map[broker.TopicPartition]*grpc.ClientConn),
grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.msg_client"),
}
}
func (mc *MessagingClient) findBroker(tp broker.TopicPartition) (*grpc.ClientConn, error) {
for _, broker := range mc.bootstrapBrokers {
grpcConnection, err := pb.GrpcDial(context.Background(), broker, mc.grpcDialOption)
if err != nil {
log.Printf("dial broker %s: %v", broker, err)
continue
}
defer grpcConnection.Close()
resp, err := messaging_pb.NewSeaweedMessagingClient(grpcConnection).FindBroker(context.Background(),
&messaging_pb.FindBrokerRequest{
Namespace: tp.Namespace,
Topic: tp.Topic,
Parition: tp.Partition,
})
if err != nil {
return nil, err
}
targetBroker := resp.Broker
return pb.GrpcDial(context.Background(), targetBroker, mc.grpcDialOption)
}
return nil, fmt.Errorf("no broker found for %+v", tp)
}

View File

@@ -0,0 +1,96 @@
package msgclient
import (
"io"
"log"
"time"
"github.com/chrislusf/seaweedfs/weed/messaging/broker"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
)
type PubChannel struct {
client messaging_pb.SeaweedMessaging_PublishClient
}
func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) {
tp := broker.TopicPartition{
Namespace: "chan",
Topic: chanName,
Partition: 0,
}
grpcConnection, err := mc.findBroker(tp)
if err != nil {
return nil, err
}
pc, err := setupPublisherClient(grpcConnection, tp)
if err != nil {
return nil, err
}
return &PubChannel{
client: pc,
}, nil
}
func (pc *PubChannel) Publish(m []byte) error {
return pc.client.Send(&messaging_pb.PublishRequest{
Data: &messaging_pb.Message{
Value: m,
},
})
}
func (pc *PubChannel) Close() error {
return pc.client.CloseSend()
}
type SubChannel struct {
ch chan []byte
stream messaging_pb.SeaweedMessaging_SubscribeClient
}
func (mc *MessagingClient) NewSubChannel(chanName string) (*SubChannel, error) {
tp := broker.TopicPartition{
Namespace: "chan",
Topic: chanName,
Partition: 0,
}
grpcConnection, err := mc.findBroker(tp)
if err != nil {
return nil, err
}
sc, err := setupSubscriberClient(grpcConnection, "", "chan", chanName, 0, time.Unix(0,0))
if err != nil {
return nil, err
}
t := &SubChannel{
ch: make(chan []byte),
stream: sc,
}
go func() {
for {
resp, subErr := t.stream.Recv()
if subErr == io.EOF {
return
}
if subErr != nil {
log.Printf("fail to receive from netchan %s: %v", chanName, subErr)
return
}
if resp.IsClose {
close(t.ch)
return
}
if resp.Data != nil {
t.ch <- resp.Data.Value
}
}
}()
return t, nil
}
func (sc *SubChannel) Channel() chan []byte {
return sc.ch
}

View File

@@ -0,0 +1,115 @@
package msgclient
import (
"context"
"github.com/OneOfOne/xxhash"
"github.com/chrislusf/seaweedfs/weed/messaging/broker"
"github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
)
type Publisher struct {
publishClients []messaging_pb.SeaweedMessaging_PublishClient
topicConfiguration *messaging_pb.TopicConfiguration
messageCount uint64
publisherId string
}
func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*Publisher, error) {
// read topic configuration
topicConfiguration := &messaging_pb.TopicConfiguration{
PartitionCount: 4,
}
publishClients := make([]messaging_pb.SeaweedMessaging_PublishClient, topicConfiguration.PartitionCount)
for i := 0; i < int(topicConfiguration.PartitionCount); i++ {
client, err := mc.setupPublisherClient(namespace, topic, int32(i))
if err != nil {
return nil, err
}
publishClients[i] = client
}
return &Publisher{
publishClients: publishClients,
topicConfiguration: topicConfiguration,
}, nil
}
func setupPublisherClient(grpcConnection *grpc.ClientConn, tp broker.TopicPartition) (messaging_pb.SeaweedMessaging_PublishClient, error) {
stream, err := messaging_pb.NewSeaweedMessagingClient(grpcConnection).Publish(context.Background())
if err != nil {
return nil, err
}
// send init message
err = stream.Send(&messaging_pb.PublishRequest{
Init: &messaging_pb.PublishRequest_InitMessage{
Namespace: tp.Namespace,
Topic: tp.Topic,
Partition: tp.Partition,
},
})
if err != nil {
return nil, err
}
// process init response
initResponse, err := stream.Recv()
if err != nil {
return nil, err
}
if initResponse.Redirect != nil {
// TODO follow redirection
}
if initResponse.Config != nil {
}
// setup looks for control messages
doneChan := make(chan error, 1)
go func() {
for {
in, err := stream.Recv()
if err != nil {
doneChan <- err
return
}
if in.Redirect != nil {
}
if in.Config != nil {
}
}
}()
return stream, nil
}
func (p *Publisher) Publish(m *messaging_pb.Message) error {
hashValue := p.messageCount
p.messageCount++
if p.topicConfiguration.Partitoning == messaging_pb.TopicConfiguration_NonNullKeyHash {
if m.Key != nil {
hashValue = xxhash.Checksum64(m.Key)
}
} else if p.topicConfiguration.Partitoning == messaging_pb.TopicConfiguration_KeyHash {
hashValue = xxhash.Checksum64(m.Key)
} else {
// round robin
}
idx := int(hashValue) % len(p.publishClients)
if idx < 0 {
idx += len(p.publishClients)
}
return p.publishClients[idx].Send(&messaging_pb.PublishRequest{
Data: m,
})
}
func (p *Publisher) Shutdown() {
for _, client := range p.publishClients {
client.CloseSend()
}
}

View File

@@ -0,0 +1,102 @@
package msgclient
import (
"context"
"io"
"time"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
)
type Subscriber struct {
subscriberClients []messaging_pb.SeaweedMessaging_SubscribeClient
subscriberId string
}
func (mc *MessagingClient) NewSubscriber(subscriberId, namespace, topic string, startTime time.Time) (*Subscriber, error) {
// read topic configuration
topicConfiguration := &messaging_pb.TopicConfiguration{
PartitionCount: 4,
}
subscriberClients := make([]messaging_pb.SeaweedMessaging_SubscribeClient, topicConfiguration.PartitionCount)
for i := 0; i < int(topicConfiguration.PartitionCount); i++ {
client, err := mc.setupSubscriberClient(subscriberId, namespace, topic, int32(i), startTime)
if err != nil {
return nil, err
}
subscriberClients[i] = client
}
return &Subscriber{
subscriberClients: subscriberClients,
subscriberId: subscriberId,
}, nil
}
func (mc *MessagingClient) setupSubscriberClient(subscriberId, namespace, topic string, partition int32, startTime time.Time) (messaging_pb.SeaweedMessaging_SubscribeClient, error) {
stream, newBroker, err := mc.initSubscriberClient(subscriberId, namespace, topic, partition, startTime)
if err != nil {
return client, err
}
if newBroker != nil {
}
return stream, nil
}
func setupSubscriberClient(grpcConnection *grpc.ClientConn, subscriberId string, namespace string, topic string, partition int32, startTime time.Time) (stream messaging_pb.SeaweedMessaging_SubscribeClient, err error) {
stream, err = messaging_pb.NewSeaweedMessagingClient(grpcConnection).Subscribe(context.Background())
if err != nil {
return
}
// send init message
err = stream.Send(&messaging_pb.SubscriberMessage{
Init: &messaging_pb.SubscriberMessage_InitMessage{
Namespace: namespace,
Topic: topic,
Partition: partition,
StartPosition: messaging_pb.SubscriberMessage_InitMessage_TIMESTAMP,
TimestampNs: startTime.UnixNano(),
SubscriberId: subscriberId,
},
})
if err != nil {
return
}
// process init response
initResponse, err := stream.Recv()
if err != nil {
return
}
if initResponse.Redirect != nil {
// TODO follow redirection
}
return stream, nil
}
func (s *Subscriber) doSubscribe(partition int, processFn func(m *messaging_pb.Message)) error {
for {
resp, listenErr := s.subscriberClients[partition].Recv()
if listenErr == io.EOF {
return nil
}
if listenErr != nil {
println(listenErr.Error())
return listenErr
}
processFn(resp.Data)
}
}
// Subscribe starts goroutines to process the messages
func (s *Subscriber) Subscribe(processFn func(m *messaging_pb.Message)) {
for i := 0; i < len(s.subscriberClients); i++ {
go s.doSubscribe(i, processFn)
}
}