seaweedfs/weed/messaging/broker/topic_lock.go

122 lines
2.7 KiB
Go
Raw Normal View History

2020-04-18 16:12:01 +08:00
package broker
import (
2020-04-19 15:18:32 +08:00
"fmt"
2020-04-18 16:12:01 +08:00
"sync"
"time"
2020-04-19 15:18:32 +08:00
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
2020-04-18 16:12:01 +08:00
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
)
type TopicPartition struct {
Namespace string
Topic string
Partition int32
}
2020-05-08 17:47:22 +08:00
const (
TopicPartitionFmt = "%s/%s_%02d"
2020-05-08 17:47:22 +08:00
)
func (tp *TopicPartition) String() string {
return fmt.Sprintf(TopicPartitionFmt, tp.Namespace, tp.Topic, tp.Partition)
}
2020-04-18 16:12:01 +08:00
type TopicLock struct {
sync.Mutex
2020-04-21 08:48:06 +08:00
cond *sync.Cond
2020-04-18 16:12:01 +08:00
subscriberCount int
publisherCount int
logBuffer *log_buffer.LogBuffer
}
type TopicLocks struct {
sync.Mutex
2020-04-19 15:18:32 +08:00
locks map[TopicPartition]*TopicLock
broker *MessageBroker
2020-04-18 16:12:01 +08:00
}
2020-04-19 15:18:32 +08:00
func NewTopicLocks(messageBroker *MessageBroker) *TopicLocks {
2020-04-18 16:12:01 +08:00
return &TopicLocks{
2020-04-19 15:18:32 +08:00
locks: make(map[TopicPartition]*TopicLock),
broker: messageBroker,
2020-04-18 16:12:01 +08:00
}
}
2020-04-19 15:18:32 +08:00
func (locks *TopicLocks) buildLogBuffer(tl *TopicLock, tp TopicPartition, topicConfig *messaging_pb.TopicConfiguration) *log_buffer.LogBuffer {
2020-04-18 16:12:01 +08:00
2020-04-19 15:18:32 +08:00
flushFn := func(startTime, stopTime time.Time, buf []byte) {
if topicConfig.IsTransient {
2020-04-30 18:05:34 +08:00
// return
}
2020-04-20 14:37:40 +08:00
// fmt.Printf("flushing with topic config %+v\n", topicConfig)
2020-04-19 15:18:32 +08:00
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)
}
2020-04-18 16:12:01 +08:00
}
2020-04-19 15:18:32 +08:00
logBuffer := log_buffer.NewLogBuffer(time.Minute, flushFn, func() {
tl.cond.Broadcast()
})
2020-04-18 16:12:01 +08:00
2020-04-19 15:18:32 +08:00
return logBuffer
2020-04-18 16:12:01 +08:00
}
2020-04-19 15:18:32 +08:00
func (tl *TopicLocks) RequestLock(partition TopicPartition, topicConfig *messaging_pb.TopicConfiguration, isPublisher bool) *TopicLock {
2020-04-18 16:12:01 +08:00
tl.Lock()
defer tl.Unlock()
lock, found := tl.locks[partition]
if !found {
lock = &TopicLock{}
2020-04-19 15:18:32 +08:00
lock.cond = sync.NewCond(&lock.Mutex)
2020-04-18 16:12:01 +08:00
tl.locks[partition] = lock
2020-04-19 15:18:32 +08:00
lock.logBuffer = tl.buildLogBuffer(lock, partition, topicConfig)
2020-04-18 16:12:01 +08:00
}
2020-04-19 15:18:32 +08:00
if isPublisher {
lock.publisherCount++
} else {
lock.subscriberCount++
}
return lock
2020-04-18 16:12:01 +08:00
}
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)
lock.logBuffer.Shutdown()
2020-04-18 16:12:01 +08:00
}
}
2020-05-08 17:47:22 +08:00
func (tl *TopicLocks) ListTopicPartitions() (tps []TopicPartition) {
tl.Lock()
defer tl.Unlock()
for k := range tl.locks {
tps = append(tps, k)
}
return
}