mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-10-21 09:57:24 +08:00
messaging can compile now
This commit is contained in:
113
weed/messaging/broker/broker_append.go
Normal file
113
weed/messaging/broker/broker_append.go
Normal file
@@ -0,0 +1,113 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/operation"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
func (broker *MessageBroker) appendToFile(targetFile string, topicConfig *messaging_pb.TopicConfiguration, data []byte) error {
|
||||
|
||||
assignResult, uploadResult, err2 := broker.assignAndUpload(topicConfig, data)
|
||||
if err2 != nil {
|
||||
return err2
|
||||
}
|
||||
|
||||
dir, name := util.FullPath(targetFile).DirAndName()
|
||||
|
||||
chunk := &filer_pb.FileChunk{
|
||||
FileId: assignResult.Fid,
|
||||
Offset: 0, // needs to be fixed during appending
|
||||
Size: uint64(uploadResult.Size),
|
||||
Mtime: time.Now().UnixNano(),
|
||||
ETag: uploadResult.ETag,
|
||||
IsGzipped: uploadResult.Gzip > 0,
|
||||
}
|
||||
|
||||
// append the chunk
|
||||
if err := broker.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
request := &filer_pb.AppendToEntryRequest{
|
||||
Directory: dir,
|
||||
EntryName: name,
|
||||
Chunks: []*filer_pb.FileChunk{chunk},
|
||||
}
|
||||
|
||||
_, err := client.AppendToEntry(context.Background(), request)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("append to file %v: %v", request, err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}); err != nil {
|
||||
return fmt.Errorf("append to file %v: %v", targetFile, err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) assignAndUpload(topicConfig *messaging_pb.TopicConfiguration, data []byte) (*operation.AssignResult, *operation.UploadResult, error) {
|
||||
|
||||
var assignResult = &operation.AssignResult{}
|
||||
|
||||
// assign a volume location
|
||||
if err := broker.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
request := &filer_pb.AssignVolumeRequest{
|
||||
Count: 1,
|
||||
Replication: topicConfig.Replication,
|
||||
Collection: topicConfig.Collection,
|
||||
}
|
||||
|
||||
resp, err := client.AssignVolume(context.Background(), request)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("assign volume failure %v: %v", request, err)
|
||||
return err
|
||||
}
|
||||
if resp.Error != "" {
|
||||
return fmt.Errorf("assign volume failure %v: %v", request, resp.Error)
|
||||
}
|
||||
|
||||
assignResult.Auth = security.EncodedJwt(resp.Auth)
|
||||
assignResult.Fid = resp.FileId
|
||||
assignResult.Url = resp.Url
|
||||
assignResult.PublicUrl = resp.PublicUrl
|
||||
assignResult.Count = uint64(resp.Count)
|
||||
|
||||
return nil
|
||||
}); err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// upload data
|
||||
targetUrl := fmt.Sprintf("http://%s/%s", assignResult.Url, assignResult.Fid)
|
||||
uploadResult, err := operation.UploadData(targetUrl, "", broker.option.Cipher, data, false, "", nil, assignResult.Auth)
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("upload data %s: %v", targetUrl, err)
|
||||
}
|
||||
// println("uploaded to", targetUrl)
|
||||
return assignResult, uploadResult, nil
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) (err error) {
|
||||
|
||||
for _, filer := range broker.option.Filers {
|
||||
if err = pb.WithFilerClient(filer, broker.grpcDialOption, fn); err != nil {
|
||||
glog.V(0).Infof("fail to connect to %s: %v", filer, err)
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
|
||||
}
|
15
weed/messaging/broker/broker_grpc_server.go
Normal file
15
weed/messaging/broker/broker_grpc_server.go
Normal file
@@ -0,0 +1,15 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
)
|
||||
|
||||
func (broker *MessageBroker) ConfigureTopic(c context.Context, request *messaging_pb.ConfigureTopicRequest) (*messaging_pb.ConfigureTopicResponse, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) GetTopicConfiguration(c context.Context, request *messaging_pb.GetTopicConfigurationRequest) (*messaging_pb.GetTopicConfigurationResponse, error) {
|
||||
panic("implement me")
|
||||
}
|
99
weed/messaging/broker/broker_grpc_server_publish.go
Normal file
99
weed/messaging/broker/broker_grpc_server_publish.go
Normal file
@@ -0,0 +1,99 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/filer2"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
)
|
||||
|
||||
func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_PublishServer) error {
|
||||
|
||||
// process initial request
|
||||
in, err := stream.Recv()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// TODO look it up
|
||||
topicConfig := &messaging_pb.TopicConfiguration{
|
||||
|
||||
}
|
||||
|
||||
// get lock
|
||||
tp := TopicPartition{
|
||||
Namespace: in.Init.Namespace,
|
||||
Topic: in.Init.Topic,
|
||||
Partition: in.Init.Partition,
|
||||
}
|
||||
logBuffer := broker.topicLocks.RequestPublisherLock(tp, func(startTime, stopTime time.Time, buf []byte) {
|
||||
|
||||
targetFile := fmt.Sprintf(
|
||||
"%s/%s/%s/%04d-%02d-%02d/%02d-%02d.part%02d",
|
||||
filer2.TopicsDir, tp.Namespace, tp.Topic,
|
||||
startTime.Year(), startTime.Month(), startTime.Day(), startTime.Hour(), startTime.Minute(),
|
||||
tp.Partition,
|
||||
)
|
||||
|
||||
if err := broker.appendToFile(targetFile, topicConfig, buf); err != nil {
|
||||
glog.V(0).Infof("log write failed %s: %v", targetFile, err)
|
||||
}
|
||||
|
||||
})
|
||||
defer broker.topicLocks.ReleaseLock(tp, true)
|
||||
|
||||
updatesChan := make(chan int32)
|
||||
|
||||
go func() {
|
||||
for update := range updatesChan {
|
||||
if err := stream.Send(&messaging_pb.PublishResponse{
|
||||
Config: &messaging_pb.PublishResponse_ConfigMessage{
|
||||
PartitionCount: update,
|
||||
},
|
||||
}); err != nil {
|
||||
glog.V(0).Infof("err sending publish response: %v", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
|
||||
// process each message
|
||||
for {
|
||||
in, err := stream.Recv()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if in.Data == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
m := &messaging_pb.Message{
|
||||
Timestamp: time.Now().UnixNano(),
|
||||
Key: in.Data.Key,
|
||||
Value: in.Data.Value,
|
||||
Headers: in.Data.Headers,
|
||||
}
|
||||
|
||||
data, err := proto.Marshal(m)
|
||||
if err != nil {
|
||||
glog.Errorf("marshall error: %v\n", err)
|
||||
continue
|
||||
}
|
||||
|
||||
logBuffer.AddToBuffer(in.Data.Key, data)
|
||||
|
||||
}
|
||||
}
|
88
weed/messaging/broker/broker_grpc_server_subscribe.go
Normal file
88
weed/messaging/broker/broker_grpc_server_subscribe.go
Normal file
@@ -0,0 +1,88 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_SubscribeServer) error {
|
||||
|
||||
// process initial request
|
||||
in, err := stream.Recv()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
subscriberId := in.Init.SubscriberId
|
||||
|
||||
// get lock
|
||||
tp := TopicPartition{
|
||||
Namespace: in.Init.Namespace,
|
||||
Topic: in.Init.Topic,
|
||||
Partition: in.Init.Partition,
|
||||
}
|
||||
lock := broker.topicLocks.RequestSubscriberLock(tp)
|
||||
defer broker.topicLocks.ReleaseLock(tp, false)
|
||||
cond := sync.NewCond(&lock.Mutex)
|
||||
|
||||
lastReadTime := time.Now()
|
||||
switch in.Init.StartPosition {
|
||||
case messaging_pb.SubscriberMessage_InitMessage_TIMESTAMP:
|
||||
lastReadTime = time.Unix(0, in.Init.TimestampNs)
|
||||
case messaging_pb.SubscriberMessage_InitMessage_LATEST:
|
||||
case messaging_pb.SubscriberMessage_InitMessage_EARLIEST:
|
||||
}
|
||||
|
||||
// how to process each message
|
||||
// an error returned will end the subscription
|
||||
eachMessageFn := func(m *messaging_pb.Message) error {
|
||||
err := stream.Send(&messaging_pb.BrokerMessage{
|
||||
Data: m,
|
||||
})
|
||||
if err != nil {
|
||||
glog.V(0).Infof("=> subscriber %v: %+v", subscriberId, err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// loop through all messages
|
||||
for {
|
||||
|
||||
_, buf := lock.logBuffer.ReadFromBuffer(lastReadTime)
|
||||
|
||||
for pos := 0; pos+4 < len(buf); {
|
||||
|
||||
size := util.BytesToUint32(buf[pos : pos+4])
|
||||
entryData := buf[pos+4 : pos+4+int(size)]
|
||||
|
||||
m := &messaging_pb.Message{}
|
||||
if err = proto.Unmarshal(entryData, m); err != nil {
|
||||
glog.Errorf("unexpected unmarshal messaging_pb.Message: %v", err)
|
||||
pos += 4 + int(size)
|
||||
continue
|
||||
}
|
||||
|
||||
if err = eachMessageFn(m); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
lastReadTime = time.Unix(0, m.Timestamp)
|
||||
pos += 4 + int(size)
|
||||
}
|
||||
|
||||
lock.Mutex.Lock()
|
||||
cond.Wait()
|
||||
lock.Mutex.Unlock()
|
||||
}
|
||||
|
||||
}
|
122
weed/messaging/broker/broker_server.go
Normal file
122
weed/messaging/broker/broker_server.go
Normal file
@@ -0,0 +1,122 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
||||
)
|
||||
|
||||
type MessageBrokerOption struct {
|
||||
Filers []string
|
||||
DefaultReplication string
|
||||
MaxMB int
|
||||
Port int
|
||||
Cipher bool
|
||||
}
|
||||
|
||||
type MessageBroker struct {
|
||||
option *MessageBrokerOption
|
||||
grpcDialOption grpc.DialOption
|
||||
topicLocks *TopicLocks
|
||||
}
|
||||
|
||||
func NewMessageBroker(option *MessageBrokerOption, grpcDialOption grpc.DialOption) (messageBroker *MessageBroker, err error) {
|
||||
|
||||
messageBroker = &MessageBroker{
|
||||
option: option,
|
||||
grpcDialOption: grpcDialOption,
|
||||
topicLocks: NewTopicLocks(),
|
||||
}
|
||||
|
||||
go messageBroker.loopForEver()
|
||||
|
||||
return messageBroker, nil
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) loopForEver() {
|
||||
|
||||
for {
|
||||
broker.checkPeers()
|
||||
time.Sleep(3 * time.Second)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) checkPeers() {
|
||||
|
||||
// contact a filer about masters
|
||||
var masters []string
|
||||
for _, filer := range broker.option.Filers {
|
||||
err := broker.withFilerClient(filer, func(client filer_pb.SeaweedFilerClient) error {
|
||||
resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
masters = append(masters, resp.Masters...)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
fmt.Printf("failed to read masters from %+v: %v\n", broker.option.Filers, err)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// contact each masters for filers
|
||||
var filers []string
|
||||
for _, master := range masters {
|
||||
err := broker.withMasterClient(master, func(client master_pb.SeaweedClient) error {
|
||||
resp, err := client.ListMasterClients(context.Background(), &master_pb.ListMasterClientsRequest{
|
||||
ClientType: "filer",
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
fmt.Printf("filers: %+v\n", resp.GrpcAddresses)
|
||||
filers = append(filers, resp.GrpcAddresses...)
|
||||
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
fmt.Printf("failed to list filers: %v\n", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// contact each filer about brokers
|
||||
for _, filer := range filers {
|
||||
err := broker.withFilerClient(filer, func(client filer_pb.SeaweedFilerClient) error {
|
||||
resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
masters = append(masters, resp.Masters...)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
fmt.Printf("failed to read masters from %+v: %v\n", broker.option.Filers, err)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) withFilerClient(filer string, fn func(filer_pb.SeaweedFilerClient) error) error {
|
||||
|
||||
return pb.WithFilerClient(filer, broker.grpcDialOption, fn)
|
||||
|
||||
}
|
||||
|
||||
func (broker *MessageBroker) withMasterClient(master string, fn func(client master_pb.SeaweedClient) error) error {
|
||||
|
||||
return pb.WithMasterClient(master, broker.grpcDialOption, func(client master_pb.SeaweedClient) error {
|
||||
return fn(client)
|
||||
})
|
||||
|
||||
}
|
80
weed/messaging/broker/topic_lock.go
Normal file
80
weed/messaging/broker/topic_lock.go
Normal file
@@ -0,0 +1,80 @@
|
||||
package broker
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
|
||||
)
|
||||
|
||||
type TopicPartition struct {
|
||||
Namespace string
|
||||
Topic string
|
||||
Partition int32
|
||||
}
|
||||
type TopicLock struct {
|
||||
sync.Mutex
|
||||
subscriberCount int
|
||||
publisherCount int
|
||||
logBuffer *log_buffer.LogBuffer
|
||||
}
|
||||
|
||||
type TopicLocks struct {
|
||||
sync.Mutex
|
||||
locks map[TopicPartition]*TopicLock
|
||||
}
|
||||
|
||||
func NewTopicLocks() *TopicLocks {
|
||||
return &TopicLocks{
|
||||
locks: make(map[TopicPartition]*TopicLock),
|
||||
}
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) RequestSubscriberLock(partition TopicPartition) *TopicLock {
|
||||
tl.Lock()
|
||||
defer tl.Unlock()
|
||||
|
||||
lock, found := tl.locks[partition]
|
||||
if !found {
|
||||
lock = &TopicLock{}
|
||||
tl.locks[partition] = lock
|
||||
}
|
||||
lock.subscriberCount++
|
||||
|
||||
return lock
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) RequestPublisherLock(partition TopicPartition, flushFn func(startTime, stopTime time.Time, buf []byte)) *log_buffer.LogBuffer {
|
||||
tl.Lock()
|
||||
defer tl.Unlock()
|
||||
|
||||
lock, found := tl.locks[partition]
|
||||
if !found {
|
||||
lock = &TopicLock{}
|
||||
tl.locks[partition] = lock
|
||||
}
|
||||
lock.publisherCount++
|
||||
cond := sync.NewCond(&lock.Mutex)
|
||||
lock.logBuffer = log_buffer.NewLogBuffer(time.Minute, flushFn, func() {
|
||||
cond.Broadcast()
|
||||
})
|
||||
return lock.logBuffer
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) ReleaseLock(partition TopicPartition, isPublisher bool) {
|
||||
tl.Lock()
|
||||
defer tl.Unlock()
|
||||
|
||||
lock, found := tl.locks[partition]
|
||||
if !found {
|
||||
return
|
||||
}
|
||||
if isPublisher {
|
||||
lock.publisherCount--
|
||||
} else {
|
||||
lock.subscriberCount--
|
||||
}
|
||||
if lock.subscriberCount <= 0 && lock.publisherCount <= 0 {
|
||||
delete(tl.locks, partition)
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user