aboutsummaryrefslogtreecommitdiff
path: root/weed/mq/topic
diff options
context:
space:
mode:
Diffstat (limited to 'weed/mq/topic')
-rw-r--r--weed/mq/topic/local_manager.go2
-rw-r--r--weed/mq/topic/local_partition.go69
-rw-r--r--weed/mq/topic/partition.go9
3 files changed, 61 insertions, 19 deletions
diff --git a/weed/mq/topic/local_manager.go b/weed/mq/topic/local_manager.go
index 173df090d..220dba0c8 100644
--- a/weed/mq/topic/local_manager.go
+++ b/weed/mq/topic/local_manager.go
@@ -34,7 +34,7 @@ func (manager *LocalTopicManager) AddTopicPartition(topic Topic, localPartition
localTopic.Partitions = append(localTopic.Partitions, localPartition)
}
-// GetTopic gets a topic from the local topic manager
+// GetTopicPartition gets a topic from the local topic manager
func (manager *LocalTopicManager) GetTopicPartition(topic Topic, partition Partition) *LocalPartition {
localTopic, ok := manager.topics.Get(topic.String())
if !ok {
diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go
index aa1274ff5..84602add7 100644
--- a/weed/mq/topic/local_partition.go
+++ b/weed/mq/topic/local_partition.go
@@ -2,8 +2,8 @@ package topic
import (
"fmt"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
- "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"time"
@@ -21,43 +21,76 @@ type LocalPartition struct {
Subscribers *LocalPartitionSubscribers
}
-func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition {
+var TIME_FORMAT = "2006-01-02-15-04-05"
+func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
return &LocalPartition{
Partition: partition,
isLeader: isLeader,
FollowerBrokers: followerBrokers,
- logBuffer: log_buffer.NewLogBuffer(
- fmt.Sprintf("%d/%4d-%4d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
- 2*time.Minute,
- func(startTime, stopTime time.Time, buf []byte) {
-
- },
- func() {
-
- },
- ),
+ logBuffer: log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
+ 2*time.Minute, logFlushFn, readFromDiskFn, func() {}),
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
}
}
-type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error
-
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
}
-func (p *LocalPartition) Subscribe(clientName string, startReadTime time.Time, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) {
- p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, onNoMessageFn, eachMessageFn)
+func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
+ onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
+ var processedPosition log_buffer.MessagePosition
+ var readPersistedLogErr error
+ var readInMemoryLogErr error
+ var isDone bool
+
+ for {
+ processedPosition, isDone, readPersistedLogErr = p.logBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
+ if readPersistedLogErr != nil {
+ glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
+ return readPersistedLogErr
+ }
+ if isDone {
+ return nil
+ }
+
+ startPosition = processedPosition
+ processedPosition, isDone, readInMemoryLogErr = p.logBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
+ startPosition = processedPosition
+
+ if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
+ continue
+ }
+ if readInMemoryLogErr != nil {
+ glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
+ return readInMemoryLogErr
+ }
+ if isDone {
+ return nil
+ }
+ }
+}
+
+func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
+ return p.logBuffer.GetEarliestTime()
+}
+
+func (p *LocalPartition) HasData() bool {
+ return !p.logBuffer.GetEarliestTime().IsZero()
+}
+
+func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
+ return p.logBuffer.GetEarliestPosition()
}
-func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition {
+func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
isLeader := assignment.LeaderBroker == string(self)
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
for i, followerBroker := range assignment.FollowerBrokers {
followers[i] = pb.ServerAddress(followerBroker)
}
- return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers)
+ return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers, logFlushFn, readFromDiskFn)
}
func (p *LocalPartition) closePublishers() {
diff --git a/weed/mq/topic/partition.go b/weed/mq/topic/partition.go
index ca34c2390..45b55c43b 100644
--- a/weed/mq/topic/partition.go
+++ b/weed/mq/topic/partition.go
@@ -11,6 +11,15 @@ type Partition struct {
UnixTimeNs int64 // in nanoseconds
}
+func NewPartition(rangeStart, rangeStop, ringSize int32, unixTimeNs int64) *Partition {
+ return &Partition{
+ RangeStart: rangeStart,
+ RangeStop: rangeStop,
+ RingSize: ringSize,
+ UnixTimeNs: unixTimeNs,
+ }
+}
+
func (partition Partition) Equals(other Partition) bool {
if partition.RangeStart != other.RangeStart {
return false