mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
basic pub and sub works!
This commit is contained in:
parent
788578d4ea
commit
d278b4c228
|
@ -1,13 +1,11 @@
|
|||
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"
|
||||
)
|
||||
|
@ -47,20 +45,7 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
|
|||
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)
|
||||
}
|
||||
|
||||
})
|
||||
tl := broker.topicLocks.RequestLock(tp, topicConfig, true)
|
||||
defer broker.topicLocks.ReleaseLock(tp, true)
|
||||
|
||||
updatesChan := make(chan int32)
|
||||
|
@ -78,7 +63,6 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
|
|||
}
|
||||
}()
|
||||
|
||||
|
||||
// process each message
|
||||
for {
|
||||
in, err := stream.Recv()
|
||||
|
@ -100,7 +84,7 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
|
|||
Headers: in.Data.Headers,
|
||||
}
|
||||
|
||||
println("received message:", string(in.Data.Value))
|
||||
// fmt.Printf("received: %d : %s\n", len(m.Value), string(m.Value))
|
||||
|
||||
data, err := proto.Marshal(m)
|
||||
if err != nil {
|
||||
|
@ -108,7 +92,7 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
|
|||
continue
|
||||
}
|
||||
|
||||
logBuffer.AddToBuffer(in.Data.Key, data)
|
||||
tl.logBuffer.AddToBuffer(in.Data.Key, data)
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,12 +2,12 @@ package broker
|
|||
|
||||
import (
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
@ -24,6 +24,12 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
|
|||
}
|
||||
|
||||
subscriberId := in.Init.SubscriberId
|
||||
println("+ subscriber:", subscriberId)
|
||||
defer println("- subscriber:", subscriberId)
|
||||
|
||||
// TODO look it up
|
||||
topicConfig := &messaging_pb.TopicConfiguration{
|
||||
}
|
||||
|
||||
// get lock
|
||||
tp := TopicPartition{
|
||||
|
@ -31,9 +37,8 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
|
|||
Topic: in.Init.Topic,
|
||||
Partition: in.Init.Partition,
|
||||
}
|
||||
lock := broker.topicLocks.RequestSubscriberLock(tp)
|
||||
lock := broker.topicLocks.RequestLock(tp, topicConfig, false)
|
||||
defer broker.topicLocks.ReleaseLock(tp, false)
|
||||
cond := sync.NewCond(&lock.Mutex)
|
||||
|
||||
lastReadTime := time.Now()
|
||||
switch in.Init.StartPosition {
|
||||
|
@ -65,13 +70,21 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
|
|||
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 {
|
||||
logEntry := &filer_pb.LogEntry{}
|
||||
if err = proto.Unmarshal(entryData, logEntry); err != nil {
|
||||
glog.Errorf("unexpected unmarshal messaging_pb.Message: %v", err)
|
||||
pos += 4 + int(size)
|
||||
continue
|
||||
}
|
||||
|
||||
m := &messaging_pb.Message{}
|
||||
if err = proto.Unmarshal(logEntry.Data, m); err != nil {
|
||||
glog.Errorf("unexpected unmarshal messaging_pb.Message: %v", err)
|
||||
pos += 4 + int(size)
|
||||
continue
|
||||
}
|
||||
|
||||
// fmt.Printf("sending : %d : %s\n", len(m.Value), string(m.Value))
|
||||
if err = eachMessageFn(m); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -81,7 +94,7 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
|
|||
}
|
||||
|
||||
lock.Mutex.Lock()
|
||||
cond.Wait()
|
||||
lock.cond.Wait()
|
||||
lock.Mutex.Unlock()
|
||||
}
|
||||
|
||||
|
|
|
@ -31,9 +31,10 @@ func NewMessageBroker(option *MessageBrokerOption, grpcDialOption grpc.DialOptio
|
|||
messageBroker = &MessageBroker{
|
||||
option: option,
|
||||
grpcDialOption: grpcDialOption,
|
||||
topicLocks: NewTopicLocks(),
|
||||
}
|
||||
|
||||
messageBroker.topicLocks = NewTopicLocks(messageBroker)
|
||||
|
||||
messageBroker.checkPeers()
|
||||
|
||||
// go messageBroker.loopForEver()
|
||||
|
|
|
@ -1,9 +1,13 @@
|
|||
package broker
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/filer2"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
|
||||
)
|
||||
|
||||
|
@ -14,6 +18,7 @@ type TopicPartition struct {
|
|||
}
|
||||
type TopicLock struct {
|
||||
sync.Mutex
|
||||
cond *sync.Cond
|
||||
subscriberCount int
|
||||
publisherCount int
|
||||
logBuffer *log_buffer.LogBuffer
|
||||
|
@ -21,44 +26,56 @@ type TopicLock struct {
|
|||
|
||||
type TopicLocks struct {
|
||||
sync.Mutex
|
||||
locks map[TopicPartition]*TopicLock
|
||||
locks map[TopicPartition]*TopicLock
|
||||
broker *MessageBroker
|
||||
}
|
||||
|
||||
func NewTopicLocks() *TopicLocks {
|
||||
func NewTopicLocks(messageBroker *MessageBroker) *TopicLocks {
|
||||
return &TopicLocks{
|
||||
locks: make(map[TopicPartition]*TopicLock),
|
||||
locks: make(map[TopicPartition]*TopicLock),
|
||||
broker: messageBroker,
|
||||
}
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) RequestSubscriberLock(partition TopicPartition) *TopicLock {
|
||||
tl.Lock()
|
||||
defer tl.Unlock()
|
||||
func (locks *TopicLocks) buildLogBuffer(tl *TopicLock, tp TopicPartition, topicConfig *messaging_pb.TopicConfiguration) *log_buffer.LogBuffer {
|
||||
|
||||
lock, found := tl.locks[partition]
|
||||
if !found {
|
||||
lock = &TopicLock{}
|
||||
tl.locks[partition] = lock
|
||||
flushFn := 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 := locks.broker.appendToFile(targetFile, topicConfig, buf); err != nil {
|
||||
glog.V(0).Infof("log write failed %s: %v", targetFile, err)
|
||||
}
|
||||
}
|
||||
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()
|
||||
logBuffer := log_buffer.NewLogBuffer(time.Minute, flushFn, func() {
|
||||
tl.cond.Broadcast()
|
||||
})
|
||||
return lock.logBuffer
|
||||
|
||||
return logBuffer
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) RequestLock(partition TopicPartition, topicConfig *messaging_pb.TopicConfiguration, isPublisher bool) *TopicLock {
|
||||
tl.Lock()
|
||||
defer tl.Unlock()
|
||||
|
||||
lock, found := tl.locks[partition]
|
||||
if !found {
|
||||
lock = &TopicLock{}
|
||||
lock.cond = sync.NewCond(&lock.Mutex)
|
||||
tl.locks[partition] = lock
|
||||
lock.logBuffer = tl.buildLogBuffer(lock, partition, topicConfig)
|
||||
}
|
||||
if isPublisher {
|
||||
lock.publisherCount++
|
||||
} else {
|
||||
lock.subscriberCount++
|
||||
}
|
||||
return lock
|
||||
}
|
||||
|
||||
func (tl *TopicLocks) ReleaseLock(partition TopicPartition, isPublisher bool) {
|
||||
|
|
Loading…
Reference in a new issue